You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2019/07/15 18:34:38 UTC

[hive] branch master updated (2350662 -> 0ea9fda4)

This is an automated email from the ASF dual-hosted git repository.

jcamacho pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git.


    from 2350662  HIVE-21973: SHOW LOCKS prints the headers twice (Rajkumar Singh via Gopal V)
     new c04fa7b  HIVE-21984: Clean up TruncateTable operation and desc (Miklos Gergely, reviewed by Jesus Camacho Rodriguez)
     new 0ea9fda4 HIVE-21983: Cut DropTableDesc/Operation to drop table, view and materialized view (Miklos Gergely, reviewed by Jesus Camacho Rodriguez)

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../java/org/apache/hadoop/hive/conf/HiveConf.java | 142 ++++-----
 .../hive/ql/ddl/table/creation/DropTableDesc.java  |  34 +--
 .../ql/ddl/table/creation/DropTableOperation.java  |  88 ++----
 .../hive/ql/ddl/table/misc/TruncateTableDesc.java  |  40 +--
 .../ql/ddl/table/misc/TruncateTableOperation.java  |   9 +-
 .../DropMaterializedViewDesc.java}                 |  30 +-
 .../ql/ddl/view/DropMaterializedViewOperation.java |  70 +++++
 .../DropViewDesc.java}                             |  30 +-
 .../DropViewOperation.java}                        |  44 ++-
 .../exec/repl/bootstrap/load/table/LoadTable.java  |   3 +-
 .../hive/ql/parse/AcidExportSemanticAnalyzer.java  |   3 +-
 .../hadoop/hive/ql/parse/DDLSemanticAnalyzer.java  | 340 ++++++++++++---------
 .../hive/ql/parse/FunctionSemanticAnalyzer.java    |   2 +-
 .../hive/ql/parse/ImportSemanticAnalyzer.java      |   3 +-
 .../hive/ql/parse/MacroSemanticAnalyzer.java       |   2 +-
 .../parse/repl/load/message/DropTableHandler.java  |   6 +-
 .../hive/ql/parse/TestMacroSemanticAnalyzer.java   |   6 +-
 .../sqlstd/TestSQLStdHiveAccessControllerHS2.java  |   2 +-
 .../test/results/clientpositive/explain_ddl.q.out  |   2 +-
 .../clientpositive/temp_table_truncate.q.out       |   4 +-
 .../clientpositive/tez/explainanalyze_3.q.out      |   2 +-
 .../results/clientpositive/tez/explainuser_3.q.out |   2 +-
 .../results/clientpositive/truncate_table.q.out    |  12 +-
 23 files changed, 480 insertions(+), 396 deletions(-)
 copy ql/src/java/org/apache/hadoop/hive/ql/ddl/{misc/InsertCommitHookDesc.java => view/DropMaterializedViewDesc.java} (59%)
 create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/ddl/view/DropMaterializedViewOperation.java
 copy ql/src/java/org/apache/hadoop/hive/ql/ddl/{misc/InsertCommitHookDesc.java => view/DropViewDesc.java} (61%)
 copy ql/src/java/org/apache/hadoop/hive/ql/ddl/{table/partition/AlterTableAddPartitionOperation.java => view/DropViewOperation.java} (50%)


[hive] 01/02: HIVE-21984: Clean up TruncateTable operation and desc (Miklos Gergely, reviewed by Jesus Camacho Rodriguez)

Posted by jc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jcamacho pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git

commit c04fa7b3353d20f5cab4c9ed034b822dd11b2adb
Author: Miklos Gergely <mg...@hortonworks.com>
AuthorDate: Mon Jul 15 10:01:04 2019 -0700

    HIVE-21984: Clean up TruncateTable operation and desc (Miklos Gergely, reviewed by Jesus Camacho Rodriguez)
    
    Close apache/hive#719
---
 .../hive/ql/ddl/table/misc/TruncateTableDesc.java  |  40 ++-
 .../ql/ddl/table/misc/TruncateTableOperation.java  |   9 +-
 .../hadoop/hive/ql/parse/DDLSemanticAnalyzer.java  | 277 +++++++++++----------
 .../test/results/clientpositive/explain_ddl.q.out  |   2 +-
 .../clientpositive/temp_table_truncate.q.out       |   4 +-
 .../results/clientpositive/truncate_table.q.out    |  12 +-
 6 files changed, 179 insertions(+), 165 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/TruncateTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/TruncateTableDesc.java
index 5f970e5..cf271fc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/TruncateTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/TruncateTableDesc.java
@@ -44,11 +44,10 @@ public class TruncateTableDesc implements DDLDescWithWriteId, Serializable {
   private final Map<String, String> partSpec;
   private final ReplicationSpec replicationSpec;
   private final boolean isTransactional;
-
-  private List<Integer> columnIndexes;
-  private Path inputDir;
-  private Path outputDir;
-  private ListBucketingCtx lbCtx;
+  private final List<Integer> columnIndexes;
+  private final Path inputDir;
+  private final Path outputDir;
+  private final ListBucketingCtx lbCtx;
 
   private long writeId = 0;
 
@@ -58,14 +57,23 @@ public class TruncateTableDesc implements DDLDescWithWriteId, Serializable {
 
   public TruncateTableDesc(String tableName, Map<String, String> partSpec, ReplicationSpec replicationSpec,
       Table table) {
+    this(tableName, partSpec, replicationSpec, table, null, null, null, null);
+  }
+
+  public TruncateTableDesc(String tableName, Map<String, String> partSpec, ReplicationSpec replicationSpec,
+      Table table, List<Integer> columnIndexes, Path inputDir, Path outputDir, ListBucketingCtx lbCtx) {
     this.tableName = tableName;
     this.fullTableName = table == null ? tableName : TableName.getDbTable(table.getDbName(), table.getTableName());
     this.partSpec = partSpec;
     this.replicationSpec = replicationSpec;
     this.isTransactional = AcidUtils.isTransactionalTable(table);
+    this.columnIndexes = columnIndexes;
+    this.inputDir = inputDir;
+    this.outputDir = outputDir;
+    this.lbCtx = lbCtx;
   }
 
-  @Explain(displayName = "TableName", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  @Explain(displayName = "table name", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
   public String getTableName() {
     return tableName;
   }
@@ -75,7 +83,7 @@ public class TruncateTableDesc implements DDLDescWithWriteId, Serializable {
     return fullTableName;
   }
 
-  @Explain(displayName = "Partition Spec", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  @Explain(displayName = "partition spec", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
   public Map<String, String> getPartSpec() {
     return partSpec;
   }
@@ -88,39 +96,23 @@ public class TruncateTableDesc implements DDLDescWithWriteId, Serializable {
     return replicationSpec;
   }
 
-  @Explain(displayName = "Column Indexes")
+  @Explain(displayName = "column indexes")
   public List<Integer> getColumnIndexes() {
     return columnIndexes;
   }
 
-  public void setColumnIndexes(List<Integer> columnIndexes) {
-    this.columnIndexes = columnIndexes;
-  }
-
   public Path getInputDir() {
     return inputDir;
   }
 
-  public void setInputDir(Path inputDir) {
-    this.inputDir = inputDir;
-  }
-
   public Path getOutputDir() {
     return outputDir;
   }
 
-  public void setOutputDir(Path outputDir) {
-    this.outputDir = outputDir;
-  }
-
   public ListBucketingCtx getLbCtx() {
     return lbCtx;
   }
 
-  public void setLbCtx(ListBucketingCtx lbCtx) {
-    this.lbCtx = lbCtx;
-  }
-
   @Override
   public void setWriteId(long writeId) {
     this.writeId = writeId;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/TruncateTableOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/TruncateTableOperation.java
index d5f3885..1244380 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/TruncateTableOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/TruncateTableOperation.java
@@ -18,14 +18,12 @@
 
 package org.apache.hadoop.hive.ql.ddl.table.misc;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Map;
 
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
-import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.ddl.DDLOperation;
@@ -49,15 +47,16 @@ public class TruncateTableOperation extends DDLOperation<TruncateTableDesc> {
           desc.getOutputDir());
       truncateWork.setListBucketingCtx(desc.getLbCtx());
       truncateWork.setMapperCannotSpanPartns(true);
+
       DriverContext driverCxt = new DriverContext();
       ColumnTruncateTask taskExec = new ColumnTruncateTask();
       taskExec.initialize(context.getQueryState(), null, driverCxt, null);
       taskExec.setWork(truncateWork);
       taskExec.setQueryPlan(context.getQueryPlan());
-      Task<? extends Serializable> subtask = taskExec;
+
       int ret = taskExec.execute(driverCxt);
-      if (subtask.getException() != null) {
-        context.getTask().setException(subtask.getException());
+      if (taskExec.getException() != null) {
+        context.getTask().setException(taskExec.getException());
       }
       return ret;
     }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index ff7f9a8..c013a1a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -1488,11 +1488,29 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     String tableName = getUnescapedName((ASTNode) root.getChild(0));
 
     Table table = getTable(tableName, true);
+    checkTruncateEligibility(ast, root, tableName, table);
+
+    Map<String, String> partSpec = getPartSpec((ASTNode) root.getChild(1));
+    addTruncateTableOutputs(root, table, partSpec);
+
+    Task<?> truncateTask = null;
+
+    // Is this a truncate column command
+    ASTNode colNamesNode = (ASTNode) ast.getFirstChildWithType(HiveParser.TOK_TABCOLNAME);
+    if (colNamesNode == null) {
+      truncateTask = getTruncateTaskWithoutColumnNames(tableName, partSpec, table);
+    } else {
+      truncateTask = getTruncateTaskWithColumnNames(root, tableName, table, partSpec, colNamesNode);
+    }
+
+    rootTasks.add(truncateTask);
+  }
+
+  private void checkTruncateEligibility(ASTNode ast, ASTNode root, String tableName, Table table)
+      throws SemanticException {
     boolean isForce = ast.getFirstChildWithType(HiveParser.TOK_FORCE) != null;
-    if (!isForce) {
-      if (table.getTableType() != TableType.MANAGED_TABLE) {
-        throw new SemanticException(ErrorMsg.TRUNCATE_FOR_NON_MANAGED_TABLE.format(tableName));
-      }
+    if (!isForce && table.getTableType() != TableType.MANAGED_TABLE) {
+      throw new SemanticException(ErrorMsg.TRUNCATE_FOR_NON_MANAGED_TABLE.format(tableName));
     }
     if (table.isNonNative()) {
       throw new SemanticException(ErrorMsg.TRUNCATE_FOR_NON_NATIVE_TABLE.format(tableName)); //TODO
@@ -1500,7 +1518,10 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     if (!table.isPartitioned() && root.getChildCount() > 1) {
       throw new SemanticException(ErrorMsg.PARTSPEC_FOR_NON_PARTITIONED_TABLE.format(tableName));
     }
-    Map<String, String> partSpec = getPartSpec((ASTNode) root.getChild(1));
+  }
+
+  private void addTruncateTableOutputs(ASTNode root, Table table, Map<String, String> partSpec)
+      throws SemanticException {
     if (partSpec == null) {
       if (!table.isPartitioned()) {
         outputs.add(new WriteEntity(table, WriteEntity.WriteType.DDL_EXCLUSIVE));
@@ -1521,152 +1542,154 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         }
       }
     }
+  }
 
+  private Task<?> getTruncateTaskWithoutColumnNames(String tableName, Map<String, String> partSpec, Table table) {
     TruncateTableDesc truncateTblDesc = new TruncateTableDesc(tableName, partSpec, null, table);
     if (truncateTblDesc.mayNeedWriteId()) {
       setAcidDdlDesc(truncateTblDesc);
     }
 
     DDLWork ddlWork = new DDLWork(getInputs(), getOutputs(), truncateTblDesc);
-    Task<?> truncateTask = TaskFactory.get(ddlWork);
-
-    // Is this a truncate column command
-    List<String> columnNames = null;
-    ASTNode colNamesNode = (ASTNode) ast.getFirstChildWithType(HiveParser.TOK_TABCOLNAME);
-    if (colNamesNode != null) {
-      try {
-        columnNames = getColumnNames(colNamesNode);
-
-        // It would be possible to support this, but this is such a pointless command.
-        if (AcidUtils.isInsertOnlyTable(table.getParameters())) {
-          throw new SemanticException("Truncating MM table columns not presently supported");
-        }
-
-        List<String> bucketCols = null;
-        Class<? extends InputFormat> inputFormatClass = null;
-        boolean isArchived = false;
-        Path newTblPartLoc = null;
-        Path oldTblPartLoc = null;
-        List<FieldSchema> cols = null;
-        ListBucketingCtx lbCtx = null;
-        boolean isListBucketed = false;
-        List<String> listBucketColNames = null;
+    return TaskFactory.get(ddlWork);
+  }
 
-        if (table.isPartitioned()) {
-          Partition part = db.getPartition(table, partSpec, false);
+  private Task<?> getTruncateTaskWithColumnNames(ASTNode root, String tableName, Table table,
+      Map<String, String> partSpec, ASTNode colNamesNode) throws SemanticException {
+    try {
+      List<String> columnNames = getColumnNames(colNamesNode);
 
-          Path tabPath = table.getPath();
-          Path partPath = part.getDataLocation();
+      // It would be possible to support this, but this is such a pointless command.
+      if (AcidUtils.isInsertOnlyTable(table.getParameters())) {
+        throw new SemanticException("Truncating MM table columns not presently supported");
+      }
 
-          // if the table is in a different dfs than the partition,
-          // replace the partition's dfs with the table's dfs.
-          newTblPartLoc = new Path(tabPath.toUri().getScheme(), tabPath.toUri()
-              .getAuthority(), partPath.toUri().getPath());
+      List<String> bucketCols = null;
+      Class<? extends InputFormat> inputFormatClass = null;
+      boolean isArchived = false;
+      Path newTblPartLoc = null;
+      Path oldTblPartLoc = null;
+      List<FieldSchema> cols = null;
+      ListBucketingCtx lbCtx = null;
+      boolean isListBucketed = false;
+      List<String> listBucketColNames = null;
+
+      if (table.isPartitioned()) {
+        Partition part = db.getPartition(table, partSpec, false);
+
+        Path tabPath = table.getPath();
+        Path partPath = part.getDataLocation();
+
+        // if the table is in a different dfs than the partition,
+        // replace the partition's dfs with the table's dfs.
+        newTblPartLoc = new Path(tabPath.toUri().getScheme(), tabPath.toUri()
+            .getAuthority(), partPath.toUri().getPath());
+
+        oldTblPartLoc = partPath;
+
+        cols = part.getCols();
+        bucketCols = part.getBucketCols();
+        inputFormatClass = part.getInputFormatClass();
+        isArchived = ArchiveUtils.isArchived(part);
+        lbCtx = constructListBucketingCtx(part.getSkewedColNames(), part.getSkewedColValues(),
+            part.getSkewedColValueLocationMaps(), part.isStoredAsSubDirectories());
+        isListBucketed = part.isStoredAsSubDirectories();
+        listBucketColNames = part.getSkewedColNames();
+      } else {
+        // input and output are the same
+        oldTblPartLoc = table.getPath();
+        newTblPartLoc = table.getPath();
+        cols  = table.getCols();
+        bucketCols = table.getBucketCols();
+        inputFormatClass = table.getInputFormatClass();
+        lbCtx = constructListBucketingCtx(table.getSkewedColNames(), table.getSkewedColValues(),
+            table.getSkewedColValueLocationMaps(), table.isStoredAsSubDirectories());
+        isListBucketed = table.isStoredAsSubDirectories();
+        listBucketColNames = table.getSkewedColNames();
+      }
 
-          oldTblPartLoc = partPath;
+      // throw a HiveException for non-rcfile.
+      if (!inputFormatClass.equals(RCFileInputFormat.class)) {
+        throw new SemanticException(ErrorMsg.TRUNCATE_COLUMN_NOT_RC.getMsg());
+      }
 
-          cols = part.getCols();
-          bucketCols = part.getBucketCols();
-          inputFormatClass = part.getInputFormatClass();
-          isArchived = ArchiveUtils.isArchived(part);
-          lbCtx = constructListBucketingCtx(part.getSkewedColNames(), part.getSkewedColValues(),
-              part.getSkewedColValueLocationMaps(), part.isStoredAsSubDirectories());
-          isListBucketed = part.isStoredAsSubDirectories();
-          listBucketColNames = part.getSkewedColNames();
-        } else {
-          // input and output are the same
-          oldTblPartLoc = table.getPath();
-          newTblPartLoc = table.getPath();
-          cols  = table.getCols();
-          bucketCols = table.getBucketCols();
-          inputFormatClass = table.getInputFormatClass();
-          lbCtx = constructListBucketingCtx(table.getSkewedColNames(), table.getSkewedColValues(),
-              table.getSkewedColValueLocationMaps(), table.isStoredAsSubDirectories());
-          isListBucketed = table.isStoredAsSubDirectories();
-          listBucketColNames = table.getSkewedColNames();
-        }
+      // throw a HiveException if the table/partition is archived
+      if (isArchived) {
+        throw new SemanticException(ErrorMsg.TRUNCATE_COLUMN_ARCHIVED.getMsg());
+      }
 
-        // throw a HiveException for non-rcfile.
-        if (!inputFormatClass.equals(RCFileInputFormat.class)) {
-          throw new SemanticException(ErrorMsg.TRUNCATE_COLUMN_NOT_RC.getMsg());
+      Set<Integer> columnIndexes = new HashSet<Integer>();
+      for (String columnName : columnNames) {
+        boolean found = false;
+        for (int columnIndex = 0; columnIndex < cols.size(); columnIndex++) {
+          if (columnName.equalsIgnoreCase(cols.get(columnIndex).getName())) {
+            columnIndexes.add(columnIndex);
+            found = true;
+            break;
+          }
         }
-
-        // throw a HiveException if the table/partition is archived
-        if (isArchived) {
-          throw new SemanticException(ErrorMsg.TRUNCATE_COLUMN_ARCHIVED.getMsg());
+        // Throw an exception if the user is trying to truncate a column which doesn't exist
+        if (!found) {
+          throw new SemanticException(ErrorMsg.INVALID_COLUMN.getMsg(columnName));
         }
-
-        Set<Integer> columnIndexes = new HashSet<Integer>();
-        for (String columnName : columnNames) {
-          boolean found = false;
-          for (int columnIndex = 0; columnIndex < cols.size(); columnIndex++) {
-            if (columnName.equalsIgnoreCase(cols.get(columnIndex).getName())) {
-              columnIndexes.add(columnIndex);
-              found = true;
-              break;
-            }
-          }
-          // Throw an exception if the user is trying to truncate a column which doesn't exist
-          if (!found) {
-            throw new SemanticException(ErrorMsg.INVALID_COLUMN.getMsg(columnName));
-          }
-          // Throw an exception if the table/partition is bucketed on one of the columns
-          for (String bucketCol : bucketCols) {
-            if (bucketCol.equalsIgnoreCase(columnName)) {
-              throw new SemanticException(ErrorMsg.TRUNCATE_BUCKETED_COLUMN.getMsg(columnName));
-            }
+        // Throw an exception if the table/partition is bucketed on one of the columns
+        for (String bucketCol : bucketCols) {
+          if (bucketCol.equalsIgnoreCase(columnName)) {
+            throw new SemanticException(ErrorMsg.TRUNCATE_BUCKETED_COLUMN.getMsg(columnName));
           }
-          if (isListBucketed) {
-            for (String listBucketCol : listBucketColNames) {
-              if (listBucketCol.equalsIgnoreCase(columnName)) {
-                throw new SemanticException(
-                    ErrorMsg.TRUNCATE_LIST_BUCKETED_COLUMN.getMsg(columnName));
-              }
+        }
+        if (isListBucketed) {
+          for (String listBucketCol : listBucketColNames) {
+            if (listBucketCol.equalsIgnoreCase(columnName)) {
+              throw new SemanticException(
+                  ErrorMsg.TRUNCATE_LIST_BUCKETED_COLUMN.getMsg(columnName));
             }
           }
         }
+      }
 
-        truncateTblDesc.setColumnIndexes(new ArrayList<Integer>(columnIndexes));
-        truncateTblDesc.setInputDir(oldTblPartLoc);
-        truncateTblDesc.setLbCtx(lbCtx);
-
-        addInputsOutputsAlterTable(tableName, partSpec, null, AlterTableType.TRUNCATE, false);
-        ddlWork.setNeedLock(true);
-        TableDesc tblDesc = Utilities.getTableDesc(table);
-        // Write the output to temporary directory and move it to the final location at the end
-        // so the operation is atomic.
-        Path queryTmpdir = ctx.getExternalTmpPath(newTblPartLoc);
-        truncateTblDesc.setOutputDir(queryTmpdir);
-        LoadTableDesc ltd = new LoadTableDesc(queryTmpdir, tblDesc,
-            partSpec == null ? new HashMap<>() : partSpec);
-        ltd.setLbCtx(lbCtx);
-        Task<MoveWork> moveTsk =
-            TaskFactory.get(new MoveWork(null, null, ltd, null, false));
-        truncateTask.addDependentTask(moveTsk);
-
-        // Recalculate the HDFS stats if auto gather stats is set
-        if (conf.getBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER)) {
-          BasicStatsWork basicStatsWork;
-          if (oldTblPartLoc.equals(newTblPartLoc)) {
-            // If we're merging to the same location, we can avoid some metastore calls
-            TableSpec tablepart = new TableSpec(this.db, conf, root);
-            basicStatsWork = new BasicStatsWork(tablepart);
-          } else {
-            basicStatsWork = new BasicStatsWork(ltd);
-          }
-          basicStatsWork.setNoStatsAggregator(true);
-          basicStatsWork.setClearAggregatorStats(true);
-          StatsWork columnStatsWork = new StatsWork(table, basicStatsWork, conf);
+      Path queryTmpdir = ctx.getExternalTmpPath(newTblPartLoc);
+      TruncateTableDesc truncateTblDesc = new TruncateTableDesc(tableName, partSpec, null, table,
+          new ArrayList<Integer>(columnIndexes), oldTblPartLoc, queryTmpdir, lbCtx);
+      if (truncateTblDesc.mayNeedWriteId()) {
+        setAcidDdlDesc(truncateTblDesc);
+      }
 
-          Task<?> statTask = TaskFactory.get(columnStatsWork);
-          moveTsk.addDependentTask(statTask);
+      DDLWork ddlWork = new DDLWork(getInputs(), getOutputs(), truncateTblDesc);
+      Task<?> truncateTask = TaskFactory.get(ddlWork);
+
+      addInputsOutputsAlterTable(tableName, partSpec, null, AlterTableType.TRUNCATE, false);
+      ddlWork.setNeedLock(true);
+      TableDesc tblDesc = Utilities.getTableDesc(table);
+      // Write the output to temporary directory and move it to the final location at the end
+      // so the operation is atomic.
+      LoadTableDesc ltd = new LoadTableDesc(queryTmpdir, tblDesc, partSpec == null ? new HashMap<>() : partSpec);
+      ltd.setLbCtx(lbCtx);
+      Task<MoveWork> moveTsk = TaskFactory.get(new MoveWork(null, null, ltd, null, false));
+      truncateTask.addDependentTask(moveTsk);
+
+      // Recalculate the HDFS stats if auto gather stats is set
+      if (conf.getBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER)) {
+        BasicStatsWork basicStatsWork;
+        if (oldTblPartLoc.equals(newTblPartLoc)) {
+          // If we're merging to the same location, we can avoid some metastore calls
+          TableSpec tablepart = new TableSpec(this.db, conf, root);
+          basicStatsWork = new BasicStatsWork(tablepart);
+        } else {
+          basicStatsWork = new BasicStatsWork(ltd);
         }
-      } catch (HiveException e) {
-        throw new SemanticException(e);
+        basicStatsWork.setNoStatsAggregator(true);
+        basicStatsWork.setClearAggregatorStats(true);
+        StatsWork columnStatsWork = new StatsWork(table, basicStatsWork, conf);
+
+        Task<?> statTask = TaskFactory.get(columnStatsWork);
+        moveTsk.addDependentTask(statTask);
       }
-    }
 
-    rootTasks.add(truncateTask);
+      return truncateTask;
+    } catch (HiveException e) {
+      throw new SemanticException(e);
+    }
   }
 
   public static boolean isFullSpec(Table table, Map<String, String> partSpec) {
diff --git a/ql/src/test/results/clientpositive/explain_ddl.q.out b/ql/src/test/results/clientpositive/explain_ddl.q.out
index 45c8339..e0ec1c5 100644
--- a/ql/src/test/results/clientpositive/explain_ddl.q.out
+++ b/ql/src/test/results/clientpositive/explain_ddl.q.out
@@ -795,7 +795,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-0
     Truncate Table or Partition
-      TableName: M1
+      table name: M1
 
 PREHOOK: query: select count(*) from M1 where key > 0
 PREHOOK: type: QUERY
diff --git a/ql/src/test/results/clientpositive/temp_table_truncate.q.out b/ql/src/test/results/clientpositive/temp_table_truncate.q.out
index 7c470f4..0fa4f96 100644
--- a/ql/src/test/results/clientpositive/temp_table_truncate.q.out
+++ b/ql/src/test/results/clientpositive/temp_table_truncate.q.out
@@ -82,7 +82,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-0
     Truncate Table or Partition
-      TableName: tmp_src
+      table name: tmp_src
 
 PREHOOK: query: TRUNCATE TABLE tmp_src
 PREHOOK: type: TRUNCATETABLE
@@ -111,7 +111,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-0
     Truncate Table or Partition
-      TableName: tmp_srcpart
+      table name: tmp_srcpart
 
 PREHOOK: query: TRUNCATE TABLE tmp_srcpart
 PREHOOK: type: TRUNCATETABLE
diff --git a/ql/src/test/results/clientpositive/truncate_table.q.out b/ql/src/test/results/clientpositive/truncate_table.q.out
index ba35012..a8b4cab 100644
--- a/ql/src/test/results/clientpositive/truncate_table.q.out
+++ b/ql/src/test/results/clientpositive/truncate_table.q.out
@@ -126,7 +126,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-0
     Truncate Table or Partition
-      TableName: src_truncate
+      table name: src_truncate
 
 PREHOOK: query: TRUNCATE TABLE src_truncate
 PREHOOK: type: TRUNCATETABLE
@@ -163,10 +163,10 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-0
     Truncate Table or Partition
-      Partition Spec:
+      partition spec:
         ds 2008-04-08
         hr 11
-      TableName: srcpart_truncate
+      table name: srcpart_truncate
 
 PREHOOK: query: TRUNCATE TABLE srcpart_truncate partition (ds='2008-04-08', hr='11')
 PREHOOK: type: TRUNCATETABLE
@@ -207,10 +207,10 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-0
     Truncate Table or Partition
-      Partition Spec:
+      partition spec:
         ds 
         hr 12
-      TableName: srcpart_truncate
+      table name: srcpart_truncate
 
 PREHOOK: query: TRUNCATE TABLE srcpart_truncate partition (ds, hr='12')
 PREHOOK: type: TRUNCATETABLE
@@ -259,7 +259,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-0
     Truncate Table or Partition
-      TableName: srcpart_truncate
+      table name: srcpart_truncate
 
 PREHOOK: query: TRUNCATE TABLE srcpart_truncate
 PREHOOK: type: TRUNCATETABLE


[hive] 02/02: HIVE-21983: Cut DropTableDesc/Operation to drop table, view and materialized view (Miklos Gergely, reviewed by Jesus Camacho Rodriguez)

Posted by jc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jcamacho pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git

commit 0ea9fda43d1f577bb76a2e62251f259cd2d9c4b3
Author: Miklos Gergely <mg...@hortonworks.com>
AuthorDate: Mon Jul 15 11:21:10 2019 -0700

    HIVE-21983: Cut DropTableDesc/Operation to drop table, view and materialized view (Miklos Gergely, reviewed by Jesus Camacho Rodriguez)
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java | 142 ++++++++++-----------
 .../hive/ql/ddl/table/creation/DropTableDesc.java  |  34 ++---
 .../ql/ddl/table/creation/DropTableOperation.java  |  88 +++++--------
 .../hive/ql/ddl/view/DropMaterializedViewDesc.java |  52 ++++++++
 .../ql/ddl/view/DropMaterializedViewOperation.java |  70 ++++++++++
 .../hadoop/hive/ql/ddl/view/DropViewDesc.java      |  52 ++++++++
 .../hadoop/hive/ql/ddl/view/DropViewOperation.java |  68 ++++++++++
 .../exec/repl/bootstrap/load/table/LoadTable.java  |   3 +-
 .../hive/ql/parse/AcidExportSemanticAnalyzer.java  |   3 +-
 .../hadoop/hive/ql/parse/DDLSemanticAnalyzer.java  |  63 ++++++---
 .../hive/ql/parse/FunctionSemanticAnalyzer.java    |   2 +-
 .../hive/ql/parse/ImportSemanticAnalyzer.java      |   3 +-
 .../hive/ql/parse/MacroSemanticAnalyzer.java       |   2 +-
 .../parse/repl/load/message/DropTableHandler.java  |   6 +-
 .../hive/ql/parse/TestMacroSemanticAnalyzer.java   |   6 +-
 .../sqlstd/TestSQLStdHiveAccessControllerHS2.java  |   2 +-
 .../clientpositive/tez/explainanalyze_3.q.out      |   2 +-
 .../results/clientpositive/tez/explainuser_3.q.out |   2 +-
 18 files changed, 410 insertions(+), 190 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 440d761..3e13785 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -658,7 +658,7 @@ public class HiveConf extends Configuration {
     LOCALMODEMAXINPUTFILES("hive.exec.mode.local.auto.input.files.max", 4,
         "When hive.exec.mode.local.auto is true, the number of tasks should less than this for local mode."),
 
-    DROPIGNORESNONEXISTENT("hive.exec.drop.ignorenonexistent", true,
+    DROP_IGNORES_NON_EXISTENT("hive.exec.drop.ignorenonexistent", true,
         "Do not report an error if DROP TABLE/VIEW/Index/Function specifies a non-existent table/view/function"),
 
     HIVEIGNOREMAPJOINHINT("hive.ignore.mapjoin.hint", true, "Ignore the mapjoin hint"),
@@ -5628,7 +5628,7 @@ public class HiveConf extends Configuration {
   private static String getSQLStdAuthDefaultWhiteListPattern() {
     // create the default white list from list of safe config params
     // and regex list
-    String confVarPatternStr = Joiner.on("|").join(convertVarsToRegex(sqlStdAuthSafeVarNames));
+    String confVarPatternStr = Joiner.on("|").join(convertVarsToRegex(SQL_STD_AUTH_SAFE_VAR_NAMES));
     String regexPatternStr = Joiner.on("|").join(sqlStdAuthSafeVarNameRegexes);
     return regexPatternStr + "|" + confVarPatternStr;
   }
@@ -5657,75 +5657,75 @@ public class HiveConf extends Configuration {
    * Default list of modifiable config parameters for sql standard authorization
    * For internal use only.
    */
-  private static final String [] sqlStdAuthSafeVarNames = new String [] {
-    ConfVars.AGGR_JOIN_TRANSPOSE.varname,
-    ConfVars.BYTESPERREDUCER.varname,
-    ConfVars.CLIENT_STATS_COUNTERS.varname,
-    ConfVars.DEFAULTPARTITIONNAME.varname,
-    ConfVars.DROPIGNORESNONEXISTENT.varname,
-    ConfVars.HIVECOUNTERGROUP.varname,
-    ConfVars.HIVEDEFAULTMANAGEDFILEFORMAT.varname,
-    ConfVars.HIVEENFORCEBUCKETMAPJOIN.varname,
-    ConfVars.HIVEENFORCESORTMERGEBUCKETMAPJOIN.varname,
-    ConfVars.HIVEEXPREVALUATIONCACHE.varname,
-    ConfVars.HIVEQUERYRESULTFILEFORMAT.varname,
-    ConfVars.HIVEHASHTABLELOADFACTOR.varname,
-    ConfVars.HIVEHASHTABLETHRESHOLD.varname,
-    ConfVars.HIVEIGNOREMAPJOINHINT.varname,
-    ConfVars.HIVELIMITMAXROWSIZE.varname,
-    ConfVars.HIVEMAPREDMODE.varname,
-    ConfVars.HIVEMAPSIDEAGGREGATE.varname,
-    ConfVars.HIVEOPTIMIZEMETADATAQUERIES.varname,
-    ConfVars.HIVEROWOFFSET.varname,
-    ConfVars.HIVEVARIABLESUBSTITUTE.varname,
-    ConfVars.HIVEVARIABLESUBSTITUTEDEPTH.varname,
-    ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_INCLUDEFUNCNAME.varname,
-    ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_LABEL.varname,
-    ConfVars.HIVE_CHECK_CROSS_PRODUCT.varname,
-    ConfVars.HIVE_CLI_TEZ_SESSION_ASYNC.varname,
-    ConfVars.HIVE_COMPAT.varname,
-    ConfVars.HIVE_DISPLAY_PARTITION_COLUMNS_SEPARATELY.varname,
-    ConfVars.HIVE_ERROR_ON_EMPTY_PARTITION.varname,
-    ConfVars.HIVE_EXECUTION_ENGINE.varname,
-    ConfVars.HIVE_EXEC_COPYFILE_MAXSIZE.varname,
-    ConfVars.HIVE_EXIM_URI_SCHEME_WL.varname,
-    ConfVars.HIVE_FILE_MAX_FOOTER.varname,
-    ConfVars.HIVE_INSERT_INTO_MULTILEVEL_DIRS.varname,
-    ConfVars.HIVE_LOCALIZE_RESOURCE_NUM_WAIT_ATTEMPTS.varname,
-    ConfVars.HIVE_MULTI_INSERT_MOVE_TASKS_SHARE_DEPENDENCIES.varname,
-    ConfVars.HIVE_QUERY_RESULTS_CACHE_ENABLED.varname,
-    ConfVars.HIVE_QUERY_RESULTS_CACHE_WAIT_FOR_PENDING_RESULTS.varname,
-    ConfVars.HIVE_QUOTEDID_SUPPORT.varname,
-    ConfVars.HIVE_RESULTSET_USE_UNIQUE_COLUMN_NAMES.varname,
-    ConfVars.HIVE_STATS_COLLECT_PART_LEVEL_STATS.varname,
-    ConfVars.HIVE_SCHEMA_EVOLUTION.varname,
-    ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL.varname,
-    ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_SERIALIZE_IN_TASKS.varname,
-    ConfVars.HIVE_SUPPORT_SPECICAL_CHARACTERS_IN_TABLE_NAMES.varname,
-    ConfVars.JOB_DEBUG_CAPTURE_STACKTRACES.varname,
-    ConfVars.JOB_DEBUG_TIMEOUT.varname,
-    ConfVars.LLAP_IO_ENABLED.varname,
-    ConfVars.LLAP_IO_USE_FILEID_PATH.varname,
-    ConfVars.LLAP_DAEMON_SERVICE_HOSTS.varname,
-    ConfVars.LLAP_EXECUTION_MODE.varname,
-    ConfVars.LLAP_AUTO_ALLOW_UBER.varname,
-    ConfVars.LLAP_AUTO_ENFORCE_TREE.varname,
-    ConfVars.LLAP_AUTO_ENFORCE_VECTORIZED.varname,
-    ConfVars.LLAP_AUTO_ENFORCE_STATS.varname,
-    ConfVars.LLAP_AUTO_MAX_INPUT.varname,
-    ConfVars.LLAP_AUTO_MAX_OUTPUT.varname,
-    ConfVars.LLAP_SKIP_COMPILE_UDF_CHECK.varname,
-    ConfVars.LLAP_CLIENT_CONSISTENT_SPLITS.varname,
-    ConfVars.LLAP_ENABLE_GRACE_JOIN_IN_LLAP.varname,
-    ConfVars.LLAP_ALLOW_PERMANENT_FNS.varname,
-    ConfVars.MAXCREATEDFILES.varname,
-    ConfVars.MAXREDUCERS.varname,
-    ConfVars.NWAYJOINREORDER.varname,
-    ConfVars.OUTPUT_FILE_EXTENSION.varname,
-    ConfVars.SHOW_JOB_FAIL_DEBUG_INFO.varname,
-    ConfVars.TASKLOG_DEBUG_TIMEOUT.varname,
-    ConfVars.HIVEQUERYID.varname,
-    ConfVars.HIVEQUERYTAG.varname,
+  private static final String[] SQL_STD_AUTH_SAFE_VAR_NAMES = new String[] {
+      ConfVars.AGGR_JOIN_TRANSPOSE.varname,
+      ConfVars.BYTESPERREDUCER.varname,
+      ConfVars.CLIENT_STATS_COUNTERS.varname,
+      ConfVars.DEFAULTPARTITIONNAME.varname,
+      ConfVars.DROP_IGNORES_NON_EXISTENT.varname,
+      ConfVars.HIVECOUNTERGROUP.varname,
+      ConfVars.HIVEDEFAULTMANAGEDFILEFORMAT.varname,
+      ConfVars.HIVEENFORCEBUCKETMAPJOIN.varname,
+      ConfVars.HIVEENFORCESORTMERGEBUCKETMAPJOIN.varname,
+      ConfVars.HIVEEXPREVALUATIONCACHE.varname,
+      ConfVars.HIVEQUERYRESULTFILEFORMAT.varname,
+      ConfVars.HIVEHASHTABLELOADFACTOR.varname,
+      ConfVars.HIVEHASHTABLETHRESHOLD.varname,
+      ConfVars.HIVEIGNOREMAPJOINHINT.varname,
+      ConfVars.HIVELIMITMAXROWSIZE.varname,
+      ConfVars.HIVEMAPREDMODE.varname,
+      ConfVars.HIVEMAPSIDEAGGREGATE.varname,
+      ConfVars.HIVEOPTIMIZEMETADATAQUERIES.varname,
+      ConfVars.HIVEROWOFFSET.varname,
+      ConfVars.HIVEVARIABLESUBSTITUTE.varname,
+      ConfVars.HIVEVARIABLESUBSTITUTEDEPTH.varname,
+      ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_INCLUDEFUNCNAME.varname,
+      ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_LABEL.varname,
+      ConfVars.HIVE_CHECK_CROSS_PRODUCT.varname,
+      ConfVars.HIVE_CLI_TEZ_SESSION_ASYNC.varname,
+      ConfVars.HIVE_COMPAT.varname,
+      ConfVars.HIVE_DISPLAY_PARTITION_COLUMNS_SEPARATELY.varname,
+      ConfVars.HIVE_ERROR_ON_EMPTY_PARTITION.varname,
+      ConfVars.HIVE_EXECUTION_ENGINE.varname,
+      ConfVars.HIVE_EXEC_COPYFILE_MAXSIZE.varname,
+      ConfVars.HIVE_EXIM_URI_SCHEME_WL.varname,
+      ConfVars.HIVE_FILE_MAX_FOOTER.varname,
+      ConfVars.HIVE_INSERT_INTO_MULTILEVEL_DIRS.varname,
+      ConfVars.HIVE_LOCALIZE_RESOURCE_NUM_WAIT_ATTEMPTS.varname,
+      ConfVars.HIVE_MULTI_INSERT_MOVE_TASKS_SHARE_DEPENDENCIES.varname,
+      ConfVars.HIVE_QUERY_RESULTS_CACHE_ENABLED.varname,
+      ConfVars.HIVE_QUERY_RESULTS_CACHE_WAIT_FOR_PENDING_RESULTS.varname,
+      ConfVars.HIVE_QUOTEDID_SUPPORT.varname,
+      ConfVars.HIVE_RESULTSET_USE_UNIQUE_COLUMN_NAMES.varname,
+      ConfVars.HIVE_STATS_COLLECT_PART_LEVEL_STATS.varname,
+      ConfVars.HIVE_SCHEMA_EVOLUTION.varname,
+      ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL.varname,
+      ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_SERIALIZE_IN_TASKS.varname,
+      ConfVars.HIVE_SUPPORT_SPECICAL_CHARACTERS_IN_TABLE_NAMES.varname,
+      ConfVars.JOB_DEBUG_CAPTURE_STACKTRACES.varname,
+      ConfVars.JOB_DEBUG_TIMEOUT.varname,
+      ConfVars.LLAP_IO_ENABLED.varname,
+      ConfVars.LLAP_IO_USE_FILEID_PATH.varname,
+      ConfVars.LLAP_DAEMON_SERVICE_HOSTS.varname,
+      ConfVars.LLAP_EXECUTION_MODE.varname,
+      ConfVars.LLAP_AUTO_ALLOW_UBER.varname,
+      ConfVars.LLAP_AUTO_ENFORCE_TREE.varname,
+      ConfVars.LLAP_AUTO_ENFORCE_VECTORIZED.varname,
+      ConfVars.LLAP_AUTO_ENFORCE_STATS.varname,
+      ConfVars.LLAP_AUTO_MAX_INPUT.varname,
+      ConfVars.LLAP_AUTO_MAX_OUTPUT.varname,
+      ConfVars.LLAP_SKIP_COMPILE_UDF_CHECK.varname,
+      ConfVars.LLAP_CLIENT_CONSISTENT_SPLITS.varname,
+      ConfVars.LLAP_ENABLE_GRACE_JOIN_IN_LLAP.varname,
+      ConfVars.LLAP_ALLOW_PERMANENT_FNS.varname,
+      ConfVars.MAXCREATEDFILES.varname,
+      ConfVars.MAXREDUCERS.varname,
+      ConfVars.NWAYJOINREORDER.varname,
+      ConfVars.OUTPUT_FILE_EXTENSION.varname,
+      ConfVars.SHOW_JOB_FAIL_DEBUG_INFO.varname,
+      ConfVars.TASKLOG_DEBUG_TIMEOUT.varname,
+      ConfVars.HIVEQUERYID.varname,
+      ConfVars.HIVEQUERYTAG.varname,
   };
 
   /**
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/DropTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/DropTableDesc.java
index 6687a4b..e1a1fab 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/DropTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/DropTableDesc.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hive.ql.ddl.table.creation;
 
 import java.io.Serializable;
 
-import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.ql.ddl.DDLDesc;
 import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
 import org.apache.hadoop.hive.ql.plan.Explain;
@@ -34,23 +33,20 @@ public class DropTableDesc implements DDLDesc, Serializable {
   private static final long serialVersionUID = 1L;
 
   private final String tableName;
-  private final TableType expectedType;
   private final boolean ifExists;
-  private final boolean ifPurge;
+  private final boolean purge;
   private final ReplicationSpec replicationSpec;
   private final boolean validationRequired;
 
-  public DropTableDesc(String tableName, TableType expectedType, boolean ifExists, boolean ifPurge,
-      ReplicationSpec replicationSpec) {
-    this(tableName, expectedType, ifExists, ifPurge, replicationSpec, true);
+  public DropTableDesc(String tableName, boolean ifExists, boolean ifPurge, ReplicationSpec replicationSpec) {
+    this(tableName, ifExists, ifPurge, replicationSpec, true);
   }
 
-  public DropTableDesc(String tableName, TableType expectedType, boolean ifExists, boolean ifPurge,
-      ReplicationSpec replicationSpec, boolean validationRequired) {
+  public DropTableDesc(String tableName, boolean ifExists, boolean purge, ReplicationSpec replicationSpec,
+      boolean validationRequired) {
     this.tableName = tableName;
-    this.expectedType = expectedType;
     this.ifExists = ifExists;
-    this.ifPurge = ifPurge;
+    this.purge = purge;
     this.replicationSpec = replicationSpec == null ? new ReplicationSpec() : replicationSpec;
     this.validationRequired = validationRequired;
   }
@@ -60,20 +56,12 @@ public class DropTableDesc implements DDLDesc, Serializable {
     return tableName;
   }
 
-  public boolean getExpectView() {
-    return expectedType != null && expectedType == TableType.VIRTUAL_VIEW;
-  }
-
-  public boolean getExpectMaterializedView() {
-    return expectedType != null && expectedType == TableType.MATERIALIZED_VIEW;
-  }
-
-  public boolean getIfExists() {
+  public boolean isIfExists() {
     return ifExists;
   }
 
-  public boolean getIfPurge() {
-    return ifPurge;
+  public boolean isPurge() {
+    return purge;
   }
 
   /**
@@ -81,10 +69,10 @@ public class DropTableDesc implements DDLDesc, Serializable {
    * This can result in a "DROP IF OLDER THAN" kind of semantic
    */
   public ReplicationSpec getReplicationSpec(){
-    return this.replicationSpec;
+    return replicationSpec;
   }
 
   public boolean getValidationRequired(){
-    return this.validationRequired;
+    return validationRequired;
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/DropTableOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/DropTableOperation.java
index 80fa4c7..8d852cd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/DropTableOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/DropTableOperation.java
@@ -18,13 +18,12 @@
 
 package org.apache.hadoop.hive.ql.ddl.table.creation;
 
-import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
 import org.apache.hadoop.hive.ql.ddl.DDLOperation;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.metadata.HiveMaterializedViewsRegistry;
 import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.PartitionIterable;
@@ -43,54 +42,25 @@ public class DropTableOperation extends DDLOperation<DropTableDesc> {
 
   @Override
   public int execute() throws HiveException {
-    Table tbl = null;
-    try {
-      tbl = context.getDb().getTable(desc.getTableName());
-    } catch (InvalidTableException e) {
-      // drop table is idempotent
+    Table table = getTable();
+    if (table == null) {
+      return 0; // dropping not existing table is handled by DDLSemanticAnalyzer
     }
 
-    // This is a true DROP TABLE
-    if (tbl != null && desc.getValidationRequired()) {
-      if (tbl.isView()) {
-        if (!desc.getExpectView()) {
-          if (desc.getIfExists()) {
-            return 0;
-          }
-          if (desc.getExpectMaterializedView()) {
-            throw new HiveException("Cannot drop a view with DROP MATERIALIZED VIEW");
-          } else {
-            throw new HiveException("Cannot drop a view with DROP TABLE");
-          }
-        }
-      } else if (tbl.isMaterializedView()) {
-        if (!desc.getExpectMaterializedView()) {
-          if (desc.getIfExists()) {
-            return 0;
-          }
-          if (desc.getExpectView()) {
-            throw new HiveException("Cannot drop a materialized view with DROP VIEW");
-          } else {
-            throw new HiveException("Cannot drop a materialized view with DROP TABLE");
-          }
-        }
-      } else {
-        if (desc.getExpectView()) {
-          if (desc.getIfExists()) {
-            return 0;
-          }
-          throw new HiveException("Cannot drop a base table with DROP VIEW");
-        } else if (desc.getExpectMaterializedView()) {
-          if (desc.getIfExists()) {
-            return 0;
-          }
-          throw new HiveException("Cannot drop a base table with DROP MATERIALIZED VIEW");
+    if (desc.getValidationRequired()) {
+      if (table.isView() || table.isMaterializedView()) {
+        if (desc.isIfExists()) {
+          return 0;
+        } else if (table.isView()) {
+          throw new HiveException("Cannot drop a view with DROP TABLE");
+        } else {
+          throw new HiveException("Cannot drop a materialized view with DROP TABLE");
         }
       }
     }
 
     ReplicationSpec replicationSpec = desc.getReplicationSpec();
-    if (tbl != null && replicationSpec.isInReplicationScope()) {
+    if (replicationSpec.isInReplicationScope()) {
       /**
        * DROP TABLE FOR REPLICATION behaves differently from DROP TABLE IF EXISTS - it more closely
        * matches a DROP TABLE IF OLDER THAN(x) semantic.
@@ -112,15 +82,15 @@ public class DropTableOperation extends DDLOperation<DropTableDesc> {
        * drop the partitions inside it that are older than this event. To wit, DROP TABLE FOR REPL
        * acts like a recursive DROP TABLE IF OLDER.
        */
-      if (!replicationSpec.allowEventReplacementInto(tbl.getParameters())) {
+      if (!replicationSpec.allowEventReplacementInto(table.getParameters())) {
         // Drop occured as part of replicating a drop, but the destination
         // table was newer than the event being replicated. Ignore, but drop
         // any partitions inside that are older.
-        if (tbl.isPartitioned()) {
-          PartitionIterable partitions = new PartitionIterable(context.getDb(), tbl, null,
-              context.getConf().getIntVar(HiveConf.ConfVars.METASTORE_BATCH_RETRIEVE_MAX));
-          for (Partition p : Iterables.filter(partitions, replicationSpec.allowEventReplacementInto())){
-            context.getDb().dropPartition(tbl.getDbName(), tbl.getTableName(), p.getValues(), true);
+        if (table.isPartitioned()) {
+          PartitionIterable partitions = new PartitionIterable(context.getDb(), table, null,
+              MetastoreConf.getIntVar(context.getConf(), MetastoreConf.ConfVars.BATCH_RETRIEVE_MAX));
+          for (Partition p : Iterables.filter(partitions, replicationSpec.allowEventReplacementInto())) {
+            context.getDb().dropPartition(table.getDbName(), table.getTableName(), p.getValues(), true);
           }
         }
         LOG.debug("DDLTask: Drop Table is skipped as table {} is newer than update", desc.getTableName());
@@ -128,18 +98,18 @@ public class DropTableOperation extends DDLOperation<DropTableDesc> {
       }
     }
 
-    // drop the table
     // TODO: API w/catalog name
-    context.getDb().dropTable(desc.getTableName(), desc.getIfPurge());
-    if (tbl != null) {
-      // Remove from cache if it is a materialized view
-      if (tbl.isMaterializedView()) {
-        HiveMaterializedViewsRegistry.get().dropMaterializedView(tbl);
-      }
-      // We have already locked the table in DDLSemanticAnalyzer, don't do it again here
-      DDLUtils.addIfAbsentByName(new WriteEntity(tbl, WriteEntity.WriteType.DDL_NO_LOCK), context);
-    }
+    context.getDb().dropTable(desc.getTableName(), desc.isPurge());
+    DDLUtils.addIfAbsentByName(new WriteEntity(table, WriteEntity.WriteType.DDL_NO_LOCK), context);
 
     return 0;
   }
+
+  private Table getTable() throws HiveException {
+    try {
+      return context.getDb().getTable(desc.getTableName());
+    } catch (InvalidTableException e) {
+      return null;
+    }
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/DropMaterializedViewDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/DropMaterializedViewDesc.java
new file mode 100644
index 0000000..3f3f78d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/DropMaterializedViewDesc.java
@@ -0,0 +1,52 @@
+/*
+ * 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.hive.ql.ddl.view;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.hive.ql.ddl.DDLDesc;
+import org.apache.hadoop.hive.ql.plan.Explain;
+import org.apache.hadoop.hive.ql.plan.Explain.Level;
+
+/**
+ * DDL task description for DROP MATERIALIZED VIEW commands.
+ */
+@Explain(displayName = "Drop Materialized View", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+public class DropMaterializedViewDesc implements DDLDesc, Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final String viewName;
+  private final boolean ifExists;
+
+  public DropMaterializedViewDesc(String viewName, boolean ifExists) {
+    this.viewName = viewName;
+    this.ifExists = ifExists;
+  }
+
+  @Explain(displayName = "view name", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  public String getTableName() {
+    return viewName;
+  }
+
+  @Explain(displayName = "if exists", displayOnlyOnTrue = true,
+      explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  public boolean isIfExists() {
+    return ifExists;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/DropMaterializedViewOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/DropMaterializedViewOperation.java
new file mode 100644
index 0000000..de09a55
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/DropMaterializedViewOperation.java
@@ -0,0 +1,70 @@
+/*
+ * 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.hive.ql.ddl.view;
+
+import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.ddl.DDLOperation;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.HiveMaterializedViewsRegistry;
+import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
+import org.apache.hadoop.hive.ql.metadata.Table;
+
+/**
+ * Operation process of dropping a materialized view.
+ */
+public class DropMaterializedViewOperation extends DDLOperation<DropMaterializedViewDesc> {
+  public DropMaterializedViewOperation(DDLOperationContext context, DropMaterializedViewDesc desc) {
+    super(context, desc);
+  }
+
+  @Override
+  public int execute() throws HiveException {
+    Table table = getTable();
+    if (table == null) {
+      return 0; // dropping not existing materialized view is handled by DDLSemanticAnalyzer
+    }
+
+    if (!table.isMaterializedView()) {
+      if (desc.isIfExists()) {
+        return 0;
+      } else if (table.isView()) {
+        throw new HiveException("Cannot drop a view with DROP MATERIALIZED VIEW");
+      } else {
+        throw new HiveException("Cannot drop a base table with DROP MATERIALIZED VIEW");
+      }
+    }
+
+    // TODO: API w/catalog name
+    context.getDb().dropTable(desc.getTableName(), false);
+    HiveMaterializedViewsRegistry.get().dropMaterializedView(table);
+    DDLUtils.addIfAbsentByName(new WriteEntity(table, WriteEntity.WriteType.DDL_NO_LOCK), context);
+
+    return 0;
+  }
+
+  private Table getTable() throws HiveException {
+    try {
+      return context.getDb().getTable(desc.getTableName());
+    } catch (InvalidTableException e) {
+      return null;
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/DropViewDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/DropViewDesc.java
new file mode 100644
index 0000000..0ffb007
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/DropViewDesc.java
@@ -0,0 +1,52 @@
+/*
+ * 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.hive.ql.ddl.view;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.hive.ql.ddl.DDLDesc;
+import org.apache.hadoop.hive.ql.plan.Explain;
+import org.apache.hadoop.hive.ql.plan.Explain.Level;
+
+/**
+ * DDL task description for DROP VIEW commands.
+ */
+@Explain(displayName = "Drop View", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+public class DropViewDesc implements DDLDesc, Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final String viewName;
+  private final boolean ifExists;
+
+  public DropViewDesc(String viewName, boolean ifExists) {
+    this.viewName = viewName;
+    this.ifExists = ifExists;
+  }
+
+  @Explain(displayName = "view name", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  public String getTableName() {
+    return viewName;
+  }
+
+  @Explain(displayName = "if exists", displayOnlyOnTrue = true,
+      explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  public boolean isIfExists() {
+    return ifExists;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/DropViewOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/DropViewOperation.java
new file mode 100644
index 0000000..26f3ded
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/DropViewOperation.java
@@ -0,0 +1,68 @@
+/*
+ * 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.hive.ql.ddl.view;
+
+import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.ddl.DDLOperation;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
+import org.apache.hadoop.hive.ql.metadata.Table;
+
+/**
+ * Operation process of dropping a view.
+ */
+public class DropViewOperation extends DDLOperation<DropViewDesc> {
+  public DropViewOperation(DDLOperationContext context, DropViewDesc desc) {
+    super(context, desc);
+  }
+
+  @Override
+  public int execute() throws HiveException {
+    Table table = getTable();
+    if (table == null) {
+      return 0; // dropping not existing view is handled by DDLSemanticAnalyzer
+    }
+
+    if (!table.isView()) {
+      if (desc.isIfExists()) {
+        return 0;
+      } else if (table.isMaterializedView()) {
+        throw new HiveException("Cannot drop a materialized view with DROP VIEW");
+      } else {
+        throw new HiveException("Cannot drop a base table with DROP VIEW");
+      }
+    }
+
+    // TODO: API w/catalog name
+    context.getDb().dropTable(desc.getTableName(), false);
+    DDLUtils.addIfAbsentByName(new WriteEntity(table, WriteEntity.WriteType.DDL_NO_LOCK), context);
+
+    return 0;
+  }
+
+  private Table getTable() throws HiveException {
+    try {
+      return context.getDb().getTable(desc.getTableName());
+    } catch (InvalidTableException e) {
+      return null;
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadTable.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadTable.java
index f238ac0..02993fc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadTable.java
@@ -334,8 +334,7 @@ public class LoadTable {
 
   private Task<?> dropTableTask(Table table) {
     assert(table != null);
-    DropTableDesc dropTblDesc = new DropTableDesc(table.getFullyQualifiedName(), table.getTableType(),
-            true, false, event.replicationSpec());
+    DropTableDesc dropTblDesc = new DropTableDesc(table.getFullyQualifiedName(), true, false, event.replicationSpec());
     return TaskFactory.get(new DDLWork(new HashSet<>(), new HashSet<>(), dropTblDesc), context.hiveConf);
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java
index 2d53bae..4240270 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java
@@ -28,7 +28,6 @@ import java.util.UUID;
 
 import org.antlr.runtime.tree.Tree;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -197,7 +196,7 @@ public class AcidExportSemanticAnalyzer extends RewriteSemanticAnalyzer {
     // Now make a task to drop temp table
     // {@link DDLSemanticAnalyzer#analyzeDropTable(ASTNode ast, TableType expectedType)
     ReplicationSpec replicationSpec = new ReplicationSpec();
-    DropTableDesc dropTblDesc = new DropTableDesc(newTableName, TableType.MANAGED_TABLE, false, true, replicationSpec);
+    DropTableDesc dropTblDesc = new DropTableDesc(newTableName, false, true, replicationSpec);
     Task<DDLWork> dropTask = TaskFactory.get(new DDLWork(new HashSet<>(), new HashSet<>(), dropTblDesc), conf);
     exportTask.addDependentTask(dropTask);
     markReadEntityForUpdate();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index c013a1a..af21fcf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -148,6 +148,8 @@ import org.apache.hadoop.hive.ql.ddl.table.storage.AlterTableSkewedByDesc;
 import org.apache.hadoop.hive.ql.ddl.table.storage.AlterTableUnarchiveDesc;
 import org.apache.hadoop.hive.ql.ddl.table.partition.AlterTableAddPartitionDesc.PartitionDesc;
 import org.apache.hadoop.hive.ql.ddl.view.AlterMaterializedViewRewriteDesc;
+import org.apache.hadoop.hive.ql.ddl.view.DropMaterializedViewDesc;
+import org.apache.hadoop.hive.ql.ddl.view.DropViewDesc;
 import org.apache.hadoop.hive.ql.ddl.workloadmanagement.AlterPoolAddTriggerDesc;
 import org.apache.hadoop.hive.ql.ddl.workloadmanagement.AlterPoolDropTriggerDesc;
 import org.apache.hadoop.hive.ql.ddl.workloadmanagement.AlterResourcePlanDesc;
@@ -399,7 +401,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       break;
     }
     case HiveParser.TOK_DROPTABLE:
-      analyzeDropTable(ast, null);
+      analyzeDropTable(ast);
       break;
     case HiveParser.TOK_TRUNCATETABLE:
       analyzeTruncateTable(ast);
@@ -479,10 +481,10 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       analyzeMetastoreCheck(ast);
       break;
     case HiveParser.TOK_DROPVIEW:
-      analyzeDropTable(ast, TableType.VIRTUAL_VIEW);
+      analyzeDropView(ast);
       break;
     case HiveParser.TOK_DROP_MATERIALIZED_VIEW:
-      analyzeDropTable(ast, TableType.MATERIALIZED_VIEW);
+      analyzeDropMaterializedView(ast);
       break;
     case HiveParser.TOK_ALTERVIEW: {
       String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
@@ -1459,28 +1461,51 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), switchDatabaseDesc)));
   }
 
-
-
-  private void analyzeDropTable(ASTNode ast, TableType expectedType)
-      throws SemanticException {
+  private void analyzeDropTable(ASTNode ast) throws SemanticException {
     String tableName = getUnescapedName((ASTNode) ast.getChild(0));
     boolean ifExists = (ast.getFirstChildWithType(HiveParser.TOK_IFEXISTS) != null);
-    // we want to signal an error if the table/view doesn't exist and we're
-    // configured not to fail silently
-    boolean throwException =
-        !ifExists && !HiveConf.getBoolVar(conf, ConfVars.DROPIGNORESNONEXISTENT);
+    boolean throwException = !ifExists && !HiveConf.getBoolVar(conf, ConfVars.DROP_IGNORES_NON_EXISTENT);
+
+    Table table = getTable(tableName, throwException);
+    if (table != null) {
+      inputs.add(new ReadEntity(table));
+      outputs.add(new WriteEntity(table, WriteEntity.WriteType.DDL_EXCLUSIVE));
+    }
 
+    boolean purge = (ast.getFirstChildWithType(HiveParser.KW_PURGE) != null);
     ReplicationSpec replicationSpec = new ReplicationSpec(ast);
+    DropTableDesc dropTableDesc = new DropTableDesc(tableName, ifExists, purge, replicationSpec);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropTableDesc)));
+  }
 
-    Table tab = getTable(tableName, throwException);
-    if (tab != null) {
-      inputs.add(new ReadEntity(tab));
-      outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL_EXCLUSIVE));
+  private void analyzeDropView(ASTNode ast) throws SemanticException {
+    String viewName = getUnescapedName((ASTNode) ast.getChild(0));
+    boolean ifExists = (ast.getFirstChildWithType(HiveParser.TOK_IFEXISTS) != null);
+    boolean throwException = !ifExists && !HiveConf.getBoolVar(conf, ConfVars.DROP_IGNORES_NON_EXISTENT);
+
+    Table view = getTable(viewName, throwException);
+    if (view != null) {
+      inputs.add(new ReadEntity(view));
+      outputs.add(new WriteEntity(view, WriteEntity.WriteType.DDL_EXCLUSIVE));
     }
 
-    boolean ifPurge = (ast.getFirstChildWithType(HiveParser.KW_PURGE) != null);
-    DropTableDesc dropTblDesc = new DropTableDesc(tableName, expectedType, ifExists, ifPurge, replicationSpec);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropTblDesc)));
+    DropViewDesc dropViewDesc = new DropViewDesc(viewName, ifExists);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropViewDesc)));
+  }
+
+  private void analyzeDropMaterializedView(ASTNode ast) throws SemanticException {
+    String viewName = getUnescapedName((ASTNode) ast.getChild(0));
+    boolean ifExists = (ast.getFirstChildWithType(HiveParser.TOK_IFEXISTS) != null);
+    boolean throwException = !ifExists && !HiveConf.getBoolVar(conf, ConfVars.DROP_IGNORES_NON_EXISTENT);
+
+    Table materializedView = getTable(viewName, throwException);
+    if (materializedView != null) {
+      inputs.add(new ReadEntity(materializedView));
+      outputs.add(new WriteEntity(materializedView, WriteEntity.WriteType.DDL_EXCLUSIVE));
+    }
+
+    DropMaterializedViewDesc dropMaterializedViewDesc = new DropMaterializedViewDesc(viewName, ifExists);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropMaterializedViewDesc)));
   }
 
   private void analyzeTruncateTable(ASTNode ast) throws SemanticException {
@@ -3358,7 +3383,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       throws SemanticException {
 
     boolean ifExists = (ast.getFirstChildWithType(HiveParser.TOK_IFEXISTS) != null)
-        || HiveConf.getBoolVar(conf, ConfVars.DROPIGNORESNONEXISTENT);
+        || HiveConf.getBoolVar(conf, ConfVars.DROP_IGNORES_NON_EXISTENT);
     // If the drop has to fail on non-existent partitions, we cannot batch expressions.
     // That is because we actually have to check each separate expression for existence.
     // We could do a small optimization for the case where expr has all columns and all
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java
index 7f167a2..8dd5674 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java
@@ -98,7 +98,7 @@ public class FunctionSemanticAnalyzer extends BaseSemanticAnalyzer {
     // we want to signal an error if the function doesn't exist and we're
     // configured not to ignore this
     boolean throwException =
-      !ifExists && !HiveConf.getBoolVar(conf, ConfVars.DROPIGNORESNONEXISTENT);
+        !ifExists && !HiveConf.getBoolVar(conf, ConfVars.DROP_IGNORES_NON_EXISTENT);
 
     FunctionInfo info = FunctionRegistry.getFunctionInfo(functionName);
     if (info == null) {
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 0197762..687122a 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
@@ -548,8 +548,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
 
   private static Task<?> dropTableTask(Table table, EximUtil.SemanticAnalyzerWrapperContext x,
                                        ReplicationSpec replicationSpec) {
-    DropTableDesc dropTblDesc = new DropTableDesc(table.getTableName(), table.getTableType(),
-            true, false, replicationSpec);
+    DropTableDesc dropTblDesc = new DropTableDesc(table.getTableName(), true, false, replicationSpec);
     return TaskFactory.get(new DDLWork(x.getInputs(), x.getOutputs(), dropTblDesc), x.getConf());
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java
index 857a5af..e79512e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java
@@ -150,7 +150,7 @@ public class MacroSemanticAnalyzer extends BaseSemanticAnalyzer {
     // we want to signal an error if the function doesn't exist and we're
     // configured not to ignore this
     boolean throwException =
-      !ifExists && !HiveConf.getBoolVar(conf, ConfVars.DROPIGNORESNONEXISTENT);
+        !ifExists && !HiveConf.getBoolVar(conf, ConfVars.DROP_IGNORES_NON_EXISTENT);
 
     // Temp macros are not allowed to have qualified names.
     if (FunctionUtils.isQualifiedFunctionName(functionName)) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropTableHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropTableHandler.java
index 4a07473..6e29d61 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropTableHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropTableHandler.java
@@ -46,10 +46,8 @@ public class DropTableHandler extends AbstractMessageHandler {
       actualTblName = msg.getTable();
     }
 
-    DropTableDesc dropTableDesc = new DropTableDesc(
-        actualDbName + "." + actualTblName,
-        null, true, true, context.eventOnlyReplicationSpec(), false
-    );
+    DropTableDesc dropTableDesc = new DropTableDesc(actualDbName + "." + actualTblName, true, true,
+        context.eventOnlyReplicationSpec(), false);
     Task<DDLWork> dropTableTask = TaskFactory.get(
         new DDLWork(readEntitySet, writeEntitySet, dropTableDesc), context.hiveConf
     );
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java
index e29e778..fff0a3d 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java
@@ -68,7 +68,7 @@ public class TestMacroSemanticAnalyzer {
   }
   @Test
   public void testDropMacroExistsDoNotIgnoreErrors() throws Exception {
-    conf.setBoolVar(ConfVars.DROPIGNORESNONEXISTENT, false);
+    conf.setBoolVar(ConfVars.DROP_IGNORES_NON_EXISTENT, false);
     FunctionRegistry.registerTemporaryUDF("SOME_MACRO", GenericUDFMacro.class);
     analyze(parse("DROP TEMPORARY MACRO SOME_MACRO"));
   }
@@ -79,7 +79,7 @@ public class TestMacroSemanticAnalyzer {
   }
   @Test(expected = SemanticException.class)
   public void testDropMacroNonExistent() throws Exception {
-    conf.setBoolVar(ConfVars.DROPIGNORESNONEXISTENT, false);
+    conf.setBoolVar(ConfVars.DROP_IGNORES_NON_EXISTENT, false);
     analyze(parse("DROP TEMPORARY MACRO SOME_MACRO"));
   }
   @Test
@@ -88,7 +88,7 @@ public class TestMacroSemanticAnalyzer {
   }
   @Test
   public void testDropMacroNonExistentWithIfExistsDoNotIgnoreNonExistent() throws Exception {
-    conf.setBoolVar(ConfVars.DROPIGNORESNONEXISTENT, false);
+    conf.setBoolVar(ConfVars.DROP_IGNORES_NON_EXISTENT, false);
     analyze(parse("DROP TEMPORARY MACRO IF EXISTS SOME_MACRO"));
   }
   @Test
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerHS2.java b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerHS2.java
index 59a52a9..b087d3b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerHS2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerHS2.java
@@ -81,7 +81,7 @@ public class TestSQLStdHiveAccessControllerHS2 {
   private List<String> getSettableParams() throws SecurityException, NoSuchFieldException,
       IllegalArgumentException, IllegalAccessException {
     // get all the variable names being converted to regex in HiveConf, using reflection
-    Field varNameField = HiveConf.class.getDeclaredField("sqlStdAuthSafeVarNames");
+    Field varNameField = HiveConf.class.getDeclaredField("SQL_STD_AUTH_SAFE_VAR_NAMES");
     varNameField.setAccessible(true);
     List<String> confVarList = Arrays.asList((String[]) varNameField.get(null));
 
diff --git a/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out b/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out
index 52cde21..a0a47f8 100644
--- a/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out
@@ -644,7 +644,7 @@ PREHOOK: type: DROPVIEW
 POSTHOOK: query: explain analyze drop view v_n5
 POSTHOOK: type: DROPVIEW
 Stage-0
-  Drop Table{"table:":"v_n5"}
+  Drop View{"view name:":"v_n5"}
 
 PREHOOK: query: create view v_n5 as with cte as (select * from src  order by key limit 5)
 select * from cte
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
index 4d58f5e..7892baf 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
@@ -522,7 +522,7 @@ PREHOOK: type: DROPVIEW
 POSTHOOK: query: explain drop view v_n1
 POSTHOOK: type: DROPVIEW
 Stage-0
-  Drop Table{"table:":"v_n1"}
+  Drop View{"view name:":"v_n1"}
 
 PREHOOK: query: explain create view v_n1 as with cte as (select * from src  order by key limit 5)
 select * from cte