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

svn commit: r1593439 [3/3] - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ hbase-protocol/src/main/...

Modified: hbase/trunk/hbase-protocol/src/main/protobuf/Master.proto
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-protocol/src/main/protobuf/Master.proto?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-protocol/src/main/protobuf/Master.proto (original)
+++ hbase/trunk/hbase-protocol/src/main/protobuf/Master.proto Thu May  8 22:25:01 2014
@@ -116,6 +116,14 @@ message DeleteTableRequest {
 message DeleteTableResponse {
 }
 
+message TruncateTableRequest {
+  required TableName tableName = 1;
+  optional bool preserveSplits = 2 [default = false];
+}
+
+message TruncateTableResponse {
+}
+
 message EnableTableRequest {
   required TableName table_name = 1;
 }
@@ -415,6 +423,10 @@ service MasterService {
   rpc DeleteTable(DeleteTableRequest)
     returns(DeleteTableResponse);
 
+  /** Truncate a table */
+  rpc truncateTable(TruncateTableRequest)
+    returns(TruncateTableResponse);
+
   /** Puts the table on-line (only needed if table has been previously taken offline) */
   rpc EnableTable(EnableTableRequest)
     returns(EnableTableResponse);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java Thu May  8 22:25:01 2014
@@ -83,6 +83,28 @@ public class BaseMasterObserver implemen
   }
 
   @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 {
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java Thu May  8 22:25:01 2014
@@ -134,6 +134,52 @@ public interface MasterObserver extends 
       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.

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Thu May  8 22:25:01 2014
@@ -91,6 +91,7 @@ import org.apache.hadoop.hbase.master.ha
 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;
@@ -1325,6 +1326,21 @@ public class HMaster extends HRegionServ
   }
 
   @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 void addColumn(final TableName tableName, final HColumnDescriptor column)
       throws IOException {
     checkInitialized();

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java Thu May  8 22:25:01 2014
@@ -402,6 +402,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 {
     ObserverContext<MasterCoprocessorEnvironment> ctx = null;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java Thu May  8 22:25:01 2014
@@ -132,6 +132,8 @@ import org.apache.hadoop.hbase.protobuf.
 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.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
@@ -454,6 +456,18 @@ public class MasterRpcServices extends R
   }
 
   @Override
+  public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request)
+      throws ServiceException {
+    try {
+      master.truncateTable(ProtobufUtil.toTableName(request.getTableName()),
+        request.getPreserveSplits());
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+    return TruncateTableResponse.newBuilder().build();
+  }
+
+  @Override
   public DisableTableResponse disableTable(RpcController controller,
       DisableTableRequest request) throws ServiceException {
     try {
@@ -616,7 +630,7 @@ public class MasterRpcServices extends R
         throw new ServiceException("The procedure is not registered: "
           + desc.getSignature());
       }
-  
+
       LOG.info(master.getClientIdAuditPrefix() + " procedure request for: "
         + desc.getSignature());
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java Thu May  8 22:25:01 2014
@@ -98,6 +98,14 @@ public interface MasterServices extends 
   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

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java Thu May  8 22:25:01 2014
@@ -107,24 +107,7 @@ public class CreateTableHandler extends 
         throw new TableExistsException(tableName);
       }
 
-      // 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 (!this.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);
-      }
+      checkAndSetEnablingTable(assignmentManager, tableName);
       success = true;
     } finally {
       if (!success) {
@@ -134,6 +117,43 @@ public class CreateTableHandler extends 
     return this;
   }
 
+  static 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);
+    }
+  }
+
+  static 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);
+    }
+  }
+
   @Override
   public String toString() {
     String name = "UnknownServerName";
@@ -172,18 +192,7 @@ public class CreateTableHandler extends 
   protected void completed(final Throwable exception) {
     releaseTableLock();
     if (exception != null) {
-      // 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 {
-        this.assignmentManager.getZKTable().removeEnablingTable(
-            this.hTableDescriptor.getTableName(), false);
-      } catch (KeeperException e) {
-        // Keeper exception should not happen here
-        LOG.error("Got a keeper exception while removing the ENABLING table znode "
-            + this.hTableDescriptor.getTableName(), e);
-      }
+      removeEnablingTable(this.assignmentManager, this.hTableDescriptor.getTableName());
     }
   }
 
@@ -225,15 +234,7 @@ public class CreateTableHandler extends 
       addRegionsToMeta(this.catalogTracker, regionInfos);
 
       // 5. Trigger immediate assignment of the regions in round-robin fashion
-      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;
-      }
+      ModifyRegionUtils.assignRegions(assignmentManager, regionInfos);
     }
 
     // 6. Set table enabled flag up in zk.

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java Thu May  8 22:25:01 2014
@@ -29,10 +29,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
 import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@@ -46,6 +48,8 @@ import org.apache.zookeeper.KeeperExcept
 public class DeleteTableHandler extends TableEventHandler {
   private static final Log LOG = LogFactory.getLog(DeleteTableHandler.class);
 
+  protected HTableDescriptor hTableDescriptor = null;
+
   public DeleteTableHandler(TableName tableName, Server server,
       final MasterServices masterServices) {
     super(EventType.C_M_DELETE_TABLE, tableName, server, masterServices);
@@ -54,19 +58,11 @@ public class DeleteTableHandler extends 
   @Override
   protected void prepareWithTableLock() throws IOException {
     // The next call fails if no such table.
-    getTableDescriptor();
+    hTableDescriptor = getTableDescriptor();
   }
 
-  @Override
-  protected void handleTableOperation(List<HRegionInfo> regions)
-  throws IOException, KeeperException {
-    MasterCoprocessorHost cpHost = ((HMaster) this.server)
-        .getMasterCoprocessorHost();
-    if (cpHost != null) {
-      cpHost.preDeleteTableHandler(this.tableName);
-    }
-
-    // 1. Wait because of region in transition
+  protected void waitRegionInTransition(final List<HRegionInfo> regions)
+      throws IOException, KeeperException {
     AssignmentManager am = this.masterServices.getAssignmentManager();
     RegionStates states = am.getRegionStates();
     long waitTime = server.getConfiguration().
@@ -93,40 +89,34 @@ public class DeleteTableHandler extends 
           region.getRegionNameAsString() + " in transitions");
       }
     }
+  }
 
-    // 2. Remove regions from META
-    LOG.debug("Deleting regions from META");
-    MetaEditor.deleteRegions(this.server.getCatalogTracker(), regions);
+  @Override
+  protected void handleTableOperation(List<HRegionInfo> regions)
+      throws IOException, KeeperException {
+    MasterCoprocessorHost cpHost = ((HMaster) this.server).getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preDeleteTableHandler(this.tableName);
+    }
 
-    // 3. Move the table in /hbase/.tmp
-    MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
-    Path tempTableDir = mfs.moveTableToTemp(tableName);
+    // 1. Wait because of region in transition
+    waitRegionInTransition(regions);
 
     try {
-      // 4. Delete 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, new Path(tempTableDir, hri.getEncodedName()));
-      }
-
-      // 5. Delete table from FS (temp directory)
-      if (!fs.delete(tempTableDir, true)) {
-        LOG.error("Couldn't delete " + tempTableDir);
-      }
-
-      LOG.debug("Table '" + tableName + "' archived!");
+      // 2. Remove table from .META. and HDFS
+      removeTableData(regions);
     } finally {
-      // 6. Update table descriptor cache
+      // 3. Update table descriptor cache
       LOG.debug("Removing '" + tableName + "' descriptor.");
       this.masterServices.getTableDescriptors().remove(tableName);
 
-      // 7. Clean up regions of the table in RegionStates.
+      AssignmentManager am = this.masterServices.getAssignmentManager();
+
+      // 4. Clean up regions of the table in RegionStates.
       LOG.debug("Removing '" + tableName + "' from region states.");
-      states.tableDeleted(tableName);
+      am.getRegionStates().tableDeleted(tableName);
 
-      // 8. If entry for this table in zk, and up in AssignmentManager, remove it.
+      // 5. If entry for this table in zk, and up in AssignmentManager, remove it.
       LOG.debug("Marking '" + tableName + "' as deleted.");
       am.getZKTable().setDeletedTable(tableName);
     }
@@ -136,6 +126,40 @@ public class DeleteTableHandler extends 
     }
   }
 
+  /**
+   * Removes the table from .META. and archives the HDFS files.
+   */
+  protected 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!");
+  }
+
   @Override
   protected void releaseTableLock() {
     super.releaseTableLock();

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java?rev=1593439&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java (added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java Thu May  8 22:25:01 2014
@@ -0,0 +1,150 @@
+/**
+ *
+ * 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.util.ArrayList;
+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.TableName;
+import org.apache.hadoop.hbase.backup.HFileArchiver;
+import org.apache.hadoop.hbase.catalog.MetaEditor;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+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.RegionStates;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.ModifyRegionUtils;
+import org.apache.hadoop.hbase.util.Threads;
+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).getMasterCoprocessorHost();
+    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
+    CreateTableHandler.checkAndSetEnablingTable(assignmentManager, tableName);
+    try {
+      // 1. Create Table Descriptor
+      new FSTableDescriptors(server.getConfiguration())
+        .createTableDescriptorForTableDirectory(tempdir, this.hTableDescriptor, 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,
+        this.hTableDescriptor, 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) {
+      CreateTableHandler.removeEnablingTable(assignmentManager, tableName);
+      throw e;
+    }
+  }
+}

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java Thu May  8 22:25:01 2014
@@ -872,6 +872,22 @@ public class AccessController extends Ba
   public void postDeleteTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
       TableName tableName) throws IOException {}
 
+   @Override
+  public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName)
+      throws IOException {
+    requirePermission("truncateTable", tableName, null, null, Action.ADMIN, Action.CREATE);
+  }
+  @Override
+  public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> c,
+      TableName tableName) throws IOException {
+  }
+  @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 preModifyTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName,
       HTableDescriptor htd) throws IOException {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java Thu May  8 22:25:01 2014
@@ -365,6 +365,28 @@ public class VisibilityController extend
   }
 
   @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 preEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
       throws IOException {
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java Thu May  8 22:25:01 2014
@@ -40,6 +40,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.
@@ -187,4 +188,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;
+    }
+  }
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java Thu May  8 22:25:01 2014
@@ -112,6 +112,9 @@ public class TestAdmin {
 
   @After
   public void tearDown() throws Exception {
+    for (HTableDescriptor htd : this.admin.listTables()) {
+      TEST_UTIL.deleteTable(htd.getName());
+    }
   }
 
   @Test (timeout=300000)
@@ -345,6 +348,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");
@@ -370,7 +415,7 @@ public class TestAdmin {
            assertTrue(exceptionThrown);
        }
    }
-  
+
   /**
    * Verify schema modification takes.
    * @throws IOException

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java Thu May  8 22:25:01 2014
@@ -80,6 +80,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;
@@ -128,6 +130,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;
@@ -152,6 +156,8 @@ public class TestMasterObserver {
       postCreateTableCalled = false;
       preDeleteTableCalled = false;
       postDeleteTableCalled = false;
+      preTruncateTableCalled = false;
+      postTruncateTableCalled = false;
       preModifyTableCalled = false;
       postModifyTableCalled = false;
       preCreateNamespaceCalled = false;
@@ -194,6 +200,8 @@ public class TestMasterObserver {
       postCreateTableHandlerCalled = false;
       preDeleteTableHandlerCalled = false;
       postDeleteTableHandlerCalled = false;
+      preTruncateTableHandlerCalled = false;
+      postTruncateTableHandlerCalled = false;
       preModifyTableHandlerCalled = false;
       postModifyTableHandlerCalled = false;
       preAddColumnHandlerCalled = false;
@@ -259,6 +267,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) {
@@ -777,6 +808,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,
@@ -1069,7 +1126,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));
@@ -1299,7 +1361,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()) {
@@ -1331,15 +1393,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 =
@@ -1347,7 +1409,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());

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java Thu May  8 22:25:01 2014
@@ -373,6 +373,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 { }
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java Thu May  8 22:25:01 2014
@@ -311,6 +311,22 @@ public class TestAccessController extend
   }
 
   @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, USER_CREATE, USER_OWNER);
+    verifyDenied(truncateTable, USER_RW, USER_RO, USER_NONE);
+  }
+
+  @Test
   public void testAddColumn() throws Exception {
     final HColumnDescriptor hcd = new HColumnDescriptor("fam_new");
     AccessTestAction action = new AccessTestAction() {
@@ -589,9 +605,9 @@ public class TestAccessController extend
 
   @Test
   public void testMergeRegions() throws Exception {
-    
+
     final List<HRegion> regions = TEST_UTIL.getHBaseCluster().findRegionsForTable(TEST_TABLE.getTableName());
-    
+
     AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
@@ -1481,7 +1497,7 @@ public class TestAccessController extend
     // revoke
     revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
       Permission.Action.WRITE, Permission.Action.READ);
-    
+
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
       BlockingRpcChannel service = acl.coprocessorService(tableName.getName());

Modified: hbase/trunk/hbase-shell/src/main/ruby/hbase/admin.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-shell/src/main/ruby/hbase/admin.rb?rev=1593439&r1=1593438&r2=1593439&view=diff
==============================================================================
--- hbase/trunk/hbase-shell/src/main/ruby/hbase/admin.rb (original)
+++ hbase/trunk/hbase-shell/src/main/ruby/hbase/admin.rb Thu May  8 22:25:01 2014
@@ -334,11 +334,24 @@ module Hbase
       yield 'Disabling table...' if block_given?
       @admin.disableTable(table_name)
 
-      yield 'Dropping table...' if block_given?
-      @admin.deleteTable(table_name)
+      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)
+          yield 'Creating table...' if block_given?
+          @admin.createTable(table_description)
+        else
+          raise e
+        end
+      end
     end
 
     #----------------------------------------------------------------------------------------------
@@ -351,11 +364,24 @@ module Hbase
       yield 'Disabling table...' if block_given?
       disable(table_name)
 
-      yield 'Dropping table...' if block_given?
-      drop(table_name)
+      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)
+          yield 'Creating table with region boundaries...' if block_given?
+          @admin.createTable(table_description, splits)
+        else
+          raise e
+        end
+      end
     end
 
     #----------------------------------------------------------------------------------------------