Repository: hive
Updated Branches:
  refs/heads/master fefeb2ac1 -> bbf5eccee


http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
----------------------------------------------------------------------
diff --git 
a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
 
b/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
index 8edb50b..e5b8495 100644
--- 
a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
+++ 
b/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
@@ -27,13 +27,15 @@ public class AlterPartitionEvent extends ListenerEvent {
   private final Partition oldPart;
   private final Partition newPart;
   private final Table table;
+  private final boolean isTruncateOp;
 
-  public AlterPartitionEvent(Partition oldPart, Partition newPart, Table table,
-      boolean status, HMSHandler handler) {
+  public AlterPartitionEvent(Partition oldPart, Partition newPart, Table 
table, boolean isTruncateOp,
+                             boolean status, HMSHandler handler) {
     super(status, handler);
     this.oldPart = oldPart;
     this.newPart = newPart;
     this.table = table;
+    this.isTruncateOp = isTruncateOp;
   }
 
   /**
@@ -58,4 +60,12 @@ public class AlterPartitionEvent extends ListenerEvent {
   public Table getTable() {
     return table;
   }
+
+  /**
+   * Get the truncate table flag
+   * @return
+   */
+  public boolean getIsTruncateOp() {
+    return isTruncateOp;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java
----------------------------------------------------------------------
diff --git 
a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java
 
b/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java
index 4d6dce2..22ea513 100644
--- 
a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java
+++ 
b/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java
@@ -26,10 +26,13 @@ public class AlterTableEvent extends ListenerEvent {
 
   private final Table newTable;
   private final Table oldTable;
-  public AlterTableEvent (Table oldTable, Table newTable, boolean status, 
HMSHandler handler) {
+  private final boolean isTruncateOp;
+
+  public AlterTableEvent (Table oldTable, Table newTable, boolean 
isTruncateOp, boolean status, HMSHandler handler) {
     super (status, handler);
     this.oldTable = oldTable;
     this.newTable = newTable;
+    this.isTruncateOp = isTruncateOp;
   }
 
   /**
@@ -45,4 +48,11 @@ public class AlterTableEvent extends ListenerEvent {
   public Table getNewTable() {
     return newTable;
   }
+
+  /**
+   * @return the flag for truncate
+   */
+  public boolean getIsTruncateOp() {
+    return isTruncateOp;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterPartitionMessage.java
----------------------------------------------------------------------
diff --git 
a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterPartitionMessage.java
 
b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterPartitionMessage.java
index ed6080b..e9ed7e5 100644
--- 
a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterPartitionMessage.java
+++ 
b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterPartitionMessage.java
@@ -31,6 +31,8 @@ public abstract class AlterPartitionMessage extends 
EventMessage {
 
   public abstract String getTable();
 
+  public abstract boolean getIsTruncateOp();
+
   public abstract Map<String,String> getKeyValues();
 
   public abstract Table getTableObj() throws Exception;

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterTableMessage.java
----------------------------------------------------------------------
diff --git 
a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterTableMessage.java
 
b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterTableMessage.java
index 5487123..39a87bc 100644
--- 
a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterTableMessage.java
+++ 
b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterTableMessage.java
@@ -28,6 +28,8 @@ public abstract class AlterTableMessage extends EventMessage {
 
   public abstract String getTable();
 
+  public abstract boolean getIsTruncateOp();
+
   public abstract Table getTableObjBefore() throws Exception;
 
   public abstract Table getTableObjAfter() throws Exception;

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
----------------------------------------------------------------------
diff --git 
a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
 
b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
index 1ed7cc5..1bd52a8 100644
--- 
a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
+++ 
b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
@@ -149,9 +149,10 @@ public abstract class MessageFactory {
    * and some are not yet supported.
    * @param before The table before the alter
    * @param after The table after the alter
+   * @param isTruncateOp Flag to denote truncate table
    * @return
    */
-  public abstract AlterTableMessage buildAlterTableMessage(Table before, Table 
after);
+  public abstract AlterTableMessage buildAlterTableMessage(Table before, Table 
after, boolean isTruncateOp);
 
   /**
    * Factory method for DropTableMessage.
@@ -175,10 +176,11 @@ public abstract class MessageFactory {
    * @param table The table in which the partition is being altered
    * @param before The partition before it was altered
    * @param after The partition after it was altered
+   * @param isTruncateOp Flag to denote truncate partition
    * @return a new AlterPartitionMessage
    */
   public abstract AlterPartitionMessage buildAlterPartitionMessage(Table 
table, Partition before,
-                                                                   Partition 
after);
+                                                                   Partition 
after, boolean isTruncateOp);
 
   /**
    * Factory method for DropPartitionMessage.

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterPartitionMessage.java
----------------------------------------------------------------------
diff --git 
a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterPartitionMessage.java
 
b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterPartitionMessage.java
index dd1bf3c..bd7776c 100644
--- 
a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterPartitionMessage.java
+++ 
b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterPartitionMessage.java
@@ -37,6 +37,9 @@ public class JSONAlterPartitionMessage extends 
AlterPartitionMessage {
   String server, servicePrincipal, db, table, tableObjJson;
 
   @JsonProperty
+  String isTruncateOp;
+
+  @JsonProperty
   Long timestamp;
 
   @JsonProperty
@@ -52,11 +55,12 @@ public class JSONAlterPartitionMessage extends 
AlterPartitionMessage {
   }
 
   public JSONAlterPartitionMessage(String server, String servicePrincipal, 
Table tableObj,
-      Partition partitionObjBefore, Partition partitionObjAfter, Long 
timestamp) {
+      Partition partitionObjBefore, Partition partitionObjAfter, boolean 
isTruncateOp, Long timestamp) {
     this.server = server;
     this.servicePrincipal = servicePrincipal;
     this.db = tableObj.getDbName();
     this.table = tableObj.getTableName();
+    this.isTruncateOp = Boolean.toString(isTruncateOp);
     this.timestamp = timestamp;
     this.keyValues = JSONMessageFactory.getPartitionKeyValues(tableObj, 
partitionObjBefore);
     try {
@@ -95,6 +99,9 @@ public class JSONAlterPartitionMessage extends 
AlterPartitionMessage {
   }
 
   @Override
+  public boolean getIsTruncateOp() { return 
Boolean.parseBoolean(isTruncateOp); }
+
+  @Override
   public Map<String, String> getKeyValues() {
     return keyValues;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterTableMessage.java
----------------------------------------------------------------------
diff --git 
a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterTableMessage.java
 
b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterTableMessage.java
index 792015e..58eb1a7 100644
--- 
a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterTableMessage.java
+++ 
b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterTableMessage.java
@@ -32,6 +32,9 @@ public class JSONAlterTableMessage extends AlterTableMessage {
   String server, servicePrincipal, db, table, tableObjBeforeJson, 
tableObjAfterJson;
 
   @JsonProperty
+  String isTruncateOp;
+
+  @JsonProperty
   Long timestamp;
 
   /**
@@ -41,11 +44,12 @@ public class JSONAlterTableMessage extends 
AlterTableMessage {
   }
 
   public JSONAlterTableMessage(String server, String servicePrincipal, Table 
tableObjBefore, Table tableObjAfter,
-      Long timestamp) {
+      boolean isTruncateOp, Long timestamp) {
     this.server = server;
     this.servicePrincipal = servicePrincipal;
     this.db = tableObjBefore.getDbName();
     this.table = tableObjBefore.getTableName();
+    this.isTruncateOp = Boolean.toString(isTruncateOp);
     this.timestamp = timestamp;
     try {
       this.tableObjBeforeJson = 
JSONMessageFactory.createTableObjJson(tableObjBefore);
@@ -82,6 +86,9 @@ public class JSONAlterTableMessage extends AlterTableMessage {
   }
 
   @Override
+  public boolean getIsTruncateOp() { return 
Boolean.parseBoolean(isTruncateOp); }
+
+  @Override
   public Table getTableObjBefore() throws Exception {
     return (Table) JSONMessageFactory.getTObj(tableObjBeforeJson,Table.class);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
----------------------------------------------------------------------
diff --git 
a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
 
b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
index bb81949..04a4041 100644
--- 
a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
+++ 
b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
@@ -28,6 +28,10 @@ import javax.annotation.Nullable;
 
 import com.google.common.collect.Iterables;
 
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.Index;
@@ -104,8 +108,8 @@ public class JSONMessageFactory extends MessageFactory {
   }
 
   @Override
-  public AlterTableMessage buildAlterTableMessage(Table before, Table after) {
-    return new JSONAlterTableMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, 
before, after, now());
+  public AlterTableMessage buildAlterTableMessage(Table before, Table after, 
boolean isTruncateOp) {
+    return new JSONAlterTableMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, 
before, after, isTruncateOp, now());
   }
 
   @Override
@@ -123,8 +127,8 @@ public class JSONMessageFactory extends MessageFactory {
 
   @Override
   public AlterPartitionMessage buildAlterPartitionMessage(Table table, 
Partition before,
-      Partition after) {
-    return new JSONAlterPartitionMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, 
table, before, after,
+      Partition after, boolean isTruncateOp) {
+    return new JSONAlterPartitionMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, 
table, before, after, isTruncateOp,
         now());
   }
 
@@ -297,5 +301,4 @@ public class JSONMessageFactory extends MessageFactory {
     };
     return getTObjs(Iterables.transform(jsonArrayIterator, textExtractor), 
objClass);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 917e565..757b7fc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -4661,32 +4661,8 @@ public class DDLTask extends Task<DDLWork> implements 
Serializable {
     String tableName = truncateTableDesc.getTableName();
     Map<String, String> partSpec = truncateTableDesc.getPartSpec();
 
-    Table table = db.getTable(tableName, true);
-    boolean isAutopurge = 
"true".equalsIgnoreCase(table.getProperty("auto.purge"));
     try {
-      // this is not transactional
-      for (Path location : getLocations(db, table, partSpec)) {
-        FileSystem fs = location.getFileSystem(conf);
-        HadoopShims.HdfsEncryptionShim shim
-          = ShimLoader.getHadoopShims().createHdfsEncryptionShim(fs, conf);
-        if (!shim.isPathEncrypted(location)) {
-          HdfsUtils.HadoopFileStatus status = new 
HdfsUtils.HadoopFileStatus(conf, fs, location);
-          FileStatus targetStatus = fs.getFileStatus(location);
-          String targetGroup = targetStatus == null ? null : 
targetStatus.getGroup();
-          FileUtils.moveToTrash(fs, location, conf, isAutopurge);
-          fs.mkdirs(location);
-          HdfsUtils.setFullFileStatus(conf, status, targetGroup, fs, location, 
false);
-        } else {
-          FileStatus[] statuses = fs.listStatus(location, 
FileUtils.HIDDEN_FILES_PATH_FILTER);
-          if (statuses == null || statuses.length == 0) {
-            continue;
-          }
-          boolean success = Hive.trashFiles(fs, statuses, conf, isAutopurge);
-          if (!success) {
-            throw new HiveException("Error in deleting the contents of " + 
location.toString());
-          }
-        }
-      }
+      db.truncateTable(tableName, partSpec);
     } catch (Exception e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR);
     }
@@ -4717,58 +4693,6 @@ public class DDLTask extends Task<DDLWork> implements 
Serializable {
     return 0;
   }
 
-  private List<Path> getLocations(Hive db, Table table, Map<String, String> 
partSpec)
-      throws HiveException, InvalidOperationException {
-    List<Path> locations = new ArrayList<Path>();
-    if (partSpec == null) {
-      if (table.isPartitioned()) {
-        for (Partition partition : db.getPartitions(table)) {
-          locations.add(partition.getDataLocation());
-          EnvironmentContext environmentContext = new EnvironmentContext();
-          if (needToUpdateStats(partition.getParameters(), 
environmentContext)) {
-            db.alterPartition(table.getDbName(), table.getTableName(), 
partition, environmentContext);
-          }
-        }
-      } else {
-        locations.add(table.getPath());
-        EnvironmentContext environmentContext = new EnvironmentContext();
-        if (needToUpdateStats(table.getParameters(), environmentContext)) {
-          db.alterTable(table.getDbName()+"."+table.getTableName(), table, 
environmentContext);
-        }
-      }
-    } else {
-      for (Partition partition : db.getPartitionsByNames(table, partSpec)) {
-        locations.add(partition.getDataLocation());
-        EnvironmentContext environmentContext = new EnvironmentContext();
-        if (needToUpdateStats(partition.getParameters(), environmentContext)) {
-          db.alterPartition(table.getDbName(), table.getTableName(), 
partition, environmentContext);
-        }
-      }
-    }
-    return locations;
-  }
-
-  private boolean needToUpdateStats(Map<String,String> props, 
EnvironmentContext environmentContext) {
-    if (null == props) {
-      return false;
-    }
-    boolean statsPresent = false;
-    for (String stat : StatsSetupConst.supportedStats) {
-      String statVal = props.get(stat);
-      if (statVal != null && Long.parseLong(statVal) > 0) {
-        statsPresent = true;
-        //In the case of truncate table, we set the stats to be 0.
-        props.put(stat, "0");
-      }
-    }
-    //first set basic stats to true
-    StatsSetupConst.setBasicStatsState(props, StatsSetupConst.TRUE);
-    environmentContext.putToProperties(StatsSetupConst.STATS_GENERATED, 
StatsSetupConst.TASK);
-    //then invalidate column stats
-    StatsSetupConst.clearColumnStatsState(props);
-    return statsPresent;
-  }
-
   @Override
   public StageType getType() {
     return StageType.DDL;

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java 
b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index ec36487..45c77a2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -1201,6 +1201,27 @@ public class Hive {
     }
   }
 
+
+
+  /**
+   * Truncates the table/partition as per specifications. Just trash the data 
files
+   *
+   * @param dbDotTableName
+   *          name of the table
+   * @throws HiveException
+   */
+  public void truncateTable(String dbDotTableName, Map<String, String> 
partSpec) throws HiveException {
+    try {
+      Table table = getTable(dbDotTableName, true);
+
+      List<String> partNames = ((null == partSpec)
+                       ? null : getPartitionNames(table.getDbName(), 
table.getTableName(), partSpec, (short) -1));
+      getMSC().truncateTable(table.getDbName(), table.getTableName(), 
partNames);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
   public HiveConf getConf() {
     return (conf);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
index 8752e51..71d6074 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
@@ -505,7 +505,7 @@ public class ImportSemanticAnalyzer extends 
BaseSemanticAnalyzer {
       }
     }
 
-    // Next, we verify that the destination table is not offline, a view, or a 
non-native table
+    // Next, we verify that the destination table is not offline, or a 
non-native table
     EximUtil.validateTable(table);
 
     // If the import statement specified that we're importing to an external

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
index a85ba42..c4cc9bc 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.plan.PlanUtils;
 import org.apache.hadoop.hive.ql.plan.RenamePartitionDesc;
+import org.apache.hadoop.hive.ql.plan.TruncateTableDesc;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.IOUtils;
@@ -128,8 +129,10 @@ public class ReplicationSemanticAnalyzer extends 
BaseSemanticAnalyzer {
     EVENT_DROP_PARTITION("EVENT_DROP_PARTITION"),
     EVENT_ALTER_TABLE("EVENT_ALTER_TABLE"),
     EVENT_RENAME_TABLE("EVENT_RENAME_TABLE"),
+    EVENT_TRUNCATE_TABLE("EVENT_TRUNCATE_TABLE"),
     EVENT_ALTER_PARTITION("EVENT_ALTER_PARTITION"),
     EVENT_RENAME_PARTITION("EVENT_RENAME_PARTITION"),
+    EVENT_TRUNCATE_PARTITION("EVENT_TRUNCATE_PARTITION"),
     EVENT_INSERT("EVENT_INSERT"),
     EVENT_UNKNOWN("EVENT_UNKNOWN");
 
@@ -937,6 +940,24 @@ public class ReplicationSemanticAnalyzer extends 
BaseSemanticAnalyzer {
           }
         }
       }
+      case EVENT_TRUNCATE_TABLE: {
+        AlterTableMessage truncateTableMessage = 
md.getAlterTableMessage(dmd.getPayload());
+        String actualDbName = ((dbName == null) || dbName.isEmpty() ? 
truncateTableMessage.getDB() : dbName);
+        String actualTblName = ((tblName == null) || tblName.isEmpty() ? 
truncateTableMessage.getTable() : tblName);
+
+        TruncateTableDesc truncateTableDesc = new TruncateTableDesc(
+                actualDbName + "." + actualTblName, null);
+        Task<DDLWork> truncateTableTask = TaskFactory.get(new DDLWork(inputs, 
outputs, truncateTableDesc), conf);
+        if (precursor != null) {
+          precursor.addDependentTask(truncateTableTask);
+        }
+
+        List<Task<? extends Serializable>> tasks = new ArrayList<Task<? 
extends Serializable>>();
+        tasks.add(truncateTableTask);
+        LOG.debug("Added truncate tbl task : {}:{}", 
truncateTableTask.getId(), truncateTableDesc.getTableName());
+        dbsUpdated.put(actualDbName,dmd.getEventTo());
+        return tasks;
+      }
       case EVENT_ALTER_PARTITION: {
         return analyzeTableLoad(dbName, tblName, locn, precursor, dbsUpdated, 
tablesUpdated);
       }
@@ -978,6 +999,40 @@ public class ReplicationSemanticAnalyzer extends 
BaseSemanticAnalyzer {
         tablesUpdated.put(tableName, dmd.getEventTo());
         return tasks;
       }
+      case EVENT_TRUNCATE_PARTITION: {
+        AlterPartitionMessage truncatePtnMessage = 
md.getAlterPartitionMessage(dmd.getPayload());
+        String actualDbName = ((dbName == null) || dbName.isEmpty() ? 
truncatePtnMessage.getDB() : dbName);
+        String actualTblName = ((tblName == null) || tblName.isEmpty() ? 
truncatePtnMessage.getTable() : tblName);
+
+        Map<String, String> partSpec = new LinkedHashMap<String,String>();
+        try {
+          org.apache.hadoop.hive.metastore.api.Table tblObj = 
truncatePtnMessage.getTableObj();
+          org.apache.hadoop.hive.metastore.api.Partition pobjAfter = 
truncatePtnMessage.getPtnObjAfter();
+          Iterator<String> afterValIter = pobjAfter.getValuesIterator();
+          for (FieldSchema fs : tblObj.getPartitionKeys()){
+            partSpec.put(fs.getName(), afterValIter.next());
+          }
+        } catch (Exception e) {
+          if (!(e instanceof SemanticException)){
+            throw new SemanticException("Error reading message members", e);
+          } else {
+            throw (SemanticException)e;
+          }
+        }
+
+        TruncateTableDesc truncateTableDesc = new TruncateTableDesc(
+                actualDbName + "." + actualTblName, partSpec);
+        Task<DDLWork> truncatePtnTask = TaskFactory.get(new DDLWork(inputs, 
outputs, truncateTableDesc), conf);
+        if (precursor != null) {
+          precursor.addDependentTask(truncatePtnTask);
+        }
+
+        List<Task<? extends Serializable>> tasks = new ArrayList<Task<? 
extends Serializable>>();
+        tasks.add(truncatePtnTask);
+        LOG.debug("Added truncate ptn task : {}:{}", truncatePtnTask.getId(), 
truncateTableDesc.getTableName());
+        dbsUpdated.put(actualDbName,dmd.getEventTo());
+        return tasks;
+      }
       case EVENT_INSERT: {
         md = MessageFactory.getInstance().getDeserializer();
         InsertMessage insertMessage = md.getInsertMessage(dmd.getPayload());

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/events/AlterPartitionHandler.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/events/AlterPartitionHandler.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/events/AlterPartitionHandler.java
index 1073cd0..20d04dc 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/events/AlterPartitionHandler.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/events/AlterPartitionHandler.java
@@ -23,6 +23,7 @@ import 
org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.EximUtil;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
 
 import java.util.ArrayList;
 import java.util.Iterator;
@@ -34,6 +35,7 @@ import static 
org.apache.hadoop.hive.ql.parse.ReplicationSemanticAnalyzer.DumpMe
 public class AlterPartitionHandler extends AbstractHandler {
   private final org.apache.hadoop.hive.metastore.api.Partition after;
   private final org.apache.hadoop.hive.metastore.api.Table tableObject;
+  private final boolean isTruncateOp;
   private final Scenario scenario;
 
   AlterPartitionHandler(NotificationEvent event) throws Exception {
@@ -42,6 +44,7 @@ public class AlterPartitionHandler extends AbstractHandler {
     tableObject = apm.getTableObj();
     org.apache.hadoop.hive.metastore.api.Partition before = 
apm.getPtnObjBefore();
     after = apm.getPtnObjAfter();
+    isTruncateOp = apm.getIsTruncateOp();
     scenario = scenarioType(before, after);
   }
 
@@ -57,12 +60,18 @@ public class AlterPartitionHandler extends AbstractHandler {
       DUMPTYPE dumpType() {
         return DUMPTYPE.EVENT_RENAME_PARTITION;
       }
+    },
+    TRUNCATE {
+      @Override
+      DUMPTYPE dumpType() {
+        return DUMPTYPE.EVENT_TRUNCATE_PARTITION;
+      }
     };
 
     abstract DUMPTYPE dumpType();
   }
 
-  private static Scenario 
scenarioType(org.apache.hadoop.hive.metastore.api.Partition before,
+  private Scenario scenarioType(org.apache.hadoop.hive.metastore.api.Partition 
before,
       org.apache.hadoop.hive.metastore.api.Partition after) {
     Iterator<String> beforeValIter = before.getValuesIterator();
     Iterator<String> afterValIter = after.getValuesIterator();
@@ -71,7 +80,7 @@ public class AlterPartitionHandler extends AbstractHandler {
         return Scenario.RENAME;
       }
     }
-    return Scenario.ALTER;
+    return isTruncateOp ? Scenario.TRUNCATE : Scenario.ALTER;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/events/AlterTableHandler.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/events/AlterTableHandler.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/events/AlterTableHandler.java
index 04d9d79..bfe0181 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/events/AlterTableHandler.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/events/AlterTableHandler.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.EximUtil;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
 
 import static 
org.apache.hadoop.hive.ql.parse.ReplicationSemanticAnalyzer.DUMPTYPE;
 import static 
org.apache.hadoop.hive.ql.parse.ReplicationSemanticAnalyzer.DumpMetaData;
@@ -29,6 +30,7 @@ import static 
org.apache.hadoop.hive.ql.parse.ReplicationSemanticAnalyzer.DumpMe
 public class AlterTableHandler extends AbstractHandler {
   private final org.apache.hadoop.hive.metastore.api.Table before;
   private final org.apache.hadoop.hive.metastore.api.Table after;
+  private final boolean isTruncateOp;
   private final Scenario scenario;
 
   private enum Scenario {
@@ -43,6 +45,12 @@ public class AlterTableHandler extends AbstractHandler {
       DUMPTYPE dumpType() {
         return DUMPTYPE.EVENT_RENAME_TABLE;
       }
+    },
+    TRUNCATE {
+      @Override
+      DUMPTYPE dumpType() {
+        return DUMPTYPE.EVENT_TRUNCATE_TABLE;
+      }
     };
 
     abstract DUMPTYPE dumpType();
@@ -53,15 +61,18 @@ public class AlterTableHandler extends AbstractHandler {
     AlterTableMessage atm = 
deserializer.getAlterTableMessage(event.getMessage());
     before = atm.getTableObjBefore();
     after = atm.getTableObjAfter();
+    isTruncateOp = atm.getIsTruncateOp();
     scenario = scenarioType(before, after);
   }
 
-  private static Scenario 
scenarioType(org.apache.hadoop.hive.metastore.api.Table before,
+  private Scenario scenarioType(org.apache.hadoop.hive.metastore.api.Table 
before,
       org.apache.hadoop.hive.metastore.api.Table after) {
-    return before.getDbName().equals(after.getDbName())
-        && before.getTableName().equals(after.getTableName())
-        ? Scenario.ALTER
-        : Scenario.RENAME;
+    if (before.getDbName().equals(after.getDbName())
+        && before.getTableName().equals(after.getTableName())) {
+      return isTruncateOp ? Scenario.TRUNCATE : Scenario.ALTER;
+    } else {
+      return Scenario.RENAME;
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/bbf5ecce/ql/src/test/results/clientpositive/columnStatsUpdateForStatsOptimizer_2.q.out
----------------------------------------------------------------------
diff --git 
a/ql/src/test/results/clientpositive/columnStatsUpdateForStatsOptimizer_2.q.out 
b/ql/src/test/results/clientpositive/columnStatsUpdateForStatsOptimizer_2.q.out
index af21343..a7c9b3f 100644
--- 
a/ql/src/test/results/clientpositive/columnStatsUpdateForStatsOptimizer_2.q.out
+++ 
b/ql/src/test/results/clientpositive/columnStatsUpdateForStatsOptimizer_2.q.out
@@ -292,10 +292,10 @@ Table Type:               MANAGED_TABLE
 Table Parameters:               
        COLUMN_STATS_ACCURATE   {\"BASIC_STATS\":\"true\"}
 #### A masked pattern was here ####
-       numFiles                2                   
+       numFiles                0                   
        numRows                 0                   
        rawDataSize             0                   
-       totalSize               547                 
+       totalSize               0                   
 #### A masked pattern was here ####
                 
 # Storage Information           
@@ -322,11 +322,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: calendar
-            Statistics: Num rows: 136 Data size: 547 Basic stats: COMPLETE 
Column stats: NONE
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column 
stats: NONE
             Select Operator
               expressions: month (type: int)
               outputColumnNames: month
-              Statistics: Num rows: 136 Data size: 547 Basic stats: COMPLETE 
Column stats: NONE
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column 
stats: NONE
               Group By Operator
                 aggregations: max(month)
                 mode: hash

Reply via email to