You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/09/19 22:42:03 UTC

[17/34] hive git commit: HIVE-14249: Add simple materialized views with manual rebuilds (2) (Jesus Camacho Rodriguez, reviewed by Alan Gates)

HIVE-14249: Add simple materialized views with manual rebuilds (2) (Jesus Camacho Rodriguez, reviewed by Alan Gates)

Close apache/hive#91


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/5179558c
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/5179558c
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/5179558c

Branch: refs/heads/hive-14535
Commit: 5179558cc22649a30f127fe7ba2c5ede29af8241
Parents: 438109c
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed Jul 20 12:38:09 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu Sep 15 15:14:44 2016 +0100

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   4 +-
 .../hadoop/hive/metastore/MetaStoreUtils.java   |   7 +
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |   1 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  29 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |  16 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |  63 +++-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |  14 +-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |  12 +-
 .../hive/ql/parse/ImportSemanticAnalyzer.java   |   2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  86 ++---
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |   3 +
 .../hadoop/hive/ql/parse/TaskCompiler.java      |   5 +-
 .../hadoop/hive/ql/plan/CreateViewDesc.java     |  53 ++-
 .../hadoop/hive/ql/plan/DropTableDesc.java      |  27 +-
 .../hadoop/hive/ql/plan/HiveOperation.java      |   3 +-
 .../AuthorizationPreEventListener.java          |   2 +
 .../authorization/plugin/HiveOperationType.java |   1 +
 .../plugin/sqlstd/Operation2Privilege.java      |   2 +
 ...alized_view_authorization_rebuild_no_grant.q |  20 --
 ...erialized_view_authorization_rebuild_other.q |  14 -
 .../clientnegative/materialized_view_drop.q     |   9 +
 .../clientnegative/materialized_view_drop2.q    |   7 +
 .../test/queries/clientnegative/view_delete.q   |   1 -
 .../test/queries/clientnegative/view_update.q   |   1 -
 .../materialized_view_authorization_sqlstd.q    |   2 -
 .../clientpositive/materialized_view_describe.q |  58 ++++
 .../clientpositive/materialized_view_rebuild.q  |  13 -
 ...rialized_view_authorization_drop_other.q.out |   6 +-
 ...ized_view_authorization_no_select_perm.q.out |   4 +-
 ...ed_view_authorization_rebuild_no_grant.q.out |  42 ---
 ...lized_view_authorization_rebuild_other.q.out |  30 --
 .../materialized_view_delete.q.out              |   6 +-
 .../clientnegative/materialized_view_drop.q.out |  42 +++
 .../materialized_view_drop2.q.out               |  34 ++
 .../materialized_view_insert.q.out              |   4 +-
 .../clientnegative/materialized_view_load.q.out |   4 +-
 .../materialized_view_replace_with_view.q.out   |   6 +-
 .../materialized_view_update.q.out              |   6 +-
 .../results/clientnegative/view_delete.q.out    |   2 +-
 .../results/clientnegative/view_update.q.out    |   2 +-
 ...materialized_view_authorization_sqlstd.q.out |  34 +-
 .../materialized_view_create.q.out              |  53 ++-
 .../materialized_view_describe.q.out            | 335 +++++++++++++++++++
 .../clientpositive/materialized_view_drop.q.out |  14 +-
 .../materialized_view_rebuild.q.out             |  67 ----
 45 files changed, 769 insertions(+), 377 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
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 d68c4fb..301159e 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1110,9 +1110,9 @@ public class HiveConf extends Configuration {
         "for all tables."),
     HIVEMATERIALIZEDVIEWFILEFORMAT("hive.materializedview.fileformat", "ORC",
         new StringSet("none", "TextFile", "SequenceFile", "RCfile", "ORC"),
-        "File format for CREATE MATERIALIZED VIEW statement."),
+        "Default file format for CREATE MATERIALIZED VIEW statement"),
     HIVEMATERIALIZEDVIEWSERDE("hive.materializedview.serde",
-        "org.apache.hadoop.hive.ql.io.orc.OrcSerde", "Serde used for materialized views"),
+        "org.apache.hadoop.hive.ql.io.orc.OrcSerde", "Default SerDe used for materialized views"),
     HIVEQUERYRESULTFILEFORMAT("hive.query.result.fileformat", "SequenceFile", new StringSet("TextFile", "SequenceFile", "RCfile", "Llap"),
         "Default file format for storing result of the query."),
     HIVECHECKFILEFORMAT("hive.fileformat.check", true, "Whether to check file format or not when loading data files"),

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
index f632542..41385f7 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
@@ -1548,6 +1548,13 @@ public class MetaStoreUtils {
     return TableType.INDEX_TABLE.toString().equals(table.getTableType());
   }
 
+  public static boolean isMaterializedViewTable(Table table) {
+    if (table == null) {
+      return false;
+    }
+    return TableType.MATERIALIZED_VIEW.toString().equals(table.getTableType());
+  }
+
   /**
    * Given a map of partition column names to values, this creates a filter
    * string that can be used to call the *byFilter methods

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
index af1583f..f308832 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
@@ -453,6 +453,7 @@ public enum ErrorMsg {
   REPLACE_VIEW_WITH_MATERIALIZED(10400, "Attempt to replace view {0} with materialized view", true),
   REPLACE_MATERIALIZED_WITH_VIEW(10401, "Attempt to replace materialized view {0} with view", true),
   UPDATE_DELETE_VIEW(10402, "You cannot update or delete records in a view"),
+  MATERIALIZED_VIEW_DEF_EMPTY(10403, "Query for the materialized view rebuild could not be retrieved"),
   //========================== 20000 range starts here ========================//
   SCRIPT_INIT_ERROR(20000, "Unable to initialize custom script."),
   SCRIPT_IO_ERROR(20001, "An error occurred while reading or writing to your custom script. "

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/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 68d5fde..ec31cf4 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
@@ -3795,7 +3795,22 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
           if (dropTbl.getIfExists()) {
             return;
           }
-          throw new HiveException("Cannot drop a view with DROP TABLE");
+          if (dropTbl.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 (!dropTbl.getExpectMaterializedView()) {
+          if (dropTbl.getIfExists()) {
+            return;
+          }
+          if (dropTbl.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 (dropTbl.getExpectView()) {
@@ -3804,6 +3819,12 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
           }
           throw new HiveException(
               "Cannot drop a base table with DROP VIEW");
+        } else if (dropTbl.getExpectMaterializedView()) {
+          if (dropTbl.getIfExists()) {
+            return;
+          }
+          throw new HiveException(
+              "Cannot drop a base table with DROP MATERIALIZED VIEW");
         }
       }
     }
@@ -4247,17 +4268,21 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       if (crtView.getInputFormat() != null) {
         tbl.setInputFormatClass(crtView.getInputFormat());
       }
+
       if (crtView.getOutputFormat() != null) {
         tbl.setOutputFormatClass(crtView.getOutputFormat());
       }
 
       if (crtView.isMaterialized()) {
+        if (crtView.getLocation() != null) {
+          tbl.setDataLocation(new Path(crtView.getLocation()));
+        }
         // Short circuit the checks that the input format is valid, this is configured for all
         // materialized views and doesn't change so we don't need to check it constantly.
         tbl.getSd().setInputFormat(crtView.getInputFormat());
         tbl.getSd().setOutputFormat(crtView.getOutputFormat());
         tbl.getSd().setSerdeInfo(new SerDeInfo(crtView.getSerde(), crtView.getSerde(),
-                Collections.<String, String>emptyMap()));
+                crtView.getSerdeProps()));
       }
 
       db.createTable(tbl, crtView.getIfNotExists());

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
index dd86abd..ffb6ae3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
@@ -58,7 +58,12 @@ import org.apache.hadoop.hive.ql.hooks.LineageInfo;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.metadata.*;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.optimizer.listbucketingpruner.ListBucketingPrunerUtils;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -1001,8 +1006,7 @@ public abstract class BaseSemanticAnalyzer {
           || ast.getToken().getType() == HiveParser.TOK_TABLE_PARTITION
           || ast.getToken().getType() == HiveParser.TOK_TABTYPE
           || ast.getToken().getType() == HiveParser.TOK_CREATETABLE
-          || ast.getToken().getType() == HiveParser.TOK_CREATE_MATERIALIZED_VIEW
-          || ast.getToken().getType() == HiveParser.TOK_REBUILD_MATERIALIZED_VIEW);
+          || ast.getToken().getType() == HiveParser.TOK_CREATE_MATERIALIZED_VIEW);
       int childIndex = 0;
       numDynParts = 0;
 
@@ -1015,8 +1019,7 @@ public abstract class BaseSemanticAnalyzer {
               + tableName;
         }
         if (ast.getToken().getType() != HiveParser.TOK_CREATETABLE &&
-            ast.getToken().getType() != HiveParser.TOK_CREATE_MATERIALIZED_VIEW &&
-            ast.getToken().getType() != HiveParser.TOK_REBUILD_MATERIALIZED_VIEW) {
+            ast.getToken().getType() != HiveParser.TOK_CREATE_MATERIALIZED_VIEW) {
           tableHandle = db.getTable(tableName);
         }
       } catch (InvalidTableException ite) {
@@ -1029,8 +1032,7 @@ public abstract class BaseSemanticAnalyzer {
 
       // get partition metadata if partition specified
       if (ast.getChildCount() == 2 && ast.getToken().getType() != HiveParser.TOK_CREATETABLE &&
-          ast.getToken().getType() != HiveParser.TOK_CREATE_MATERIALIZED_VIEW &&
-          ast.getToken().getType() != HiveParser.TOK_REBUILD_MATERIALIZED_VIEW) {
+          ast.getToken().getType() != HiveParser.TOK_CREATE_MATERIALIZED_VIEW) {
         childIndex = 1;
         ASTNode partspec = (ASTNode) ast.getChild(1);
         partitions = new ArrayList<Partition>();

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index ff94160..e2ddb14 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -275,7 +275,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       // table, destination), so if the query is otherwise ok, it is as if we
       // did remove those and gave CBO the proper AST. That is kinda hacky.
       ASTNode queryForCbo = ast;
-      if (cboCtx.type == PreCboCtx.Type.CTAS) {
+      if (cboCtx.type == PreCboCtx.Type.CTAS_OR_MV) {
         queryForCbo = cboCtx.nodeOfInterest; // nodeOfInterest is the query
       }
       runCBO = canCBOHandleAst(queryForCbo, getQB(), cboCtx);
@@ -284,6 +284,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       if (runCBO) {
         disableJoinMerge = true;
         boolean reAnalyzeAST = false;
+        final boolean materializedView = getQB().isMaterializedView();
 
         try {
           if (this.conf.getBoolVar(HiveConf.ConfVars.HIVE_CBO_RETPATH_HIVEOP)) {
@@ -295,15 +296,30 @@ public class CalcitePlanner extends SemanticAnalyzer {
             // 1. Gen Optimized AST
             ASTNode newAST = getOptimizedAST();
 
-            // 1.1. Fix up the query for insert/ctas
-            newAST = fixUpCtasAndInsertAfterCbo(ast, newAST, cboCtx);
+            // 1.1. Fix up the query for insert/ctas/materialized views
+            newAST = fixUpAfterCbo(ast, newAST, cboCtx);
 
             // 2. Regen OP plan from optimized AST
             init(false);
-            if (cboCtx.type == PreCboCtx.Type.CTAS) {
-              // Redo create-table analysis, because it's not part of doPhase1.
-              setAST(newAST);
-              newAST = reAnalyzeCtasAfterCbo(newAST);
+            if (cboCtx.type == PreCboCtx.Type.CTAS_OR_MV) {
+              // Redo create-table/view analysis, because it's not part of doPhase1.
+              if (materializedView) {
+                // Use the REWRITTEN AST
+                setAST(newAST);
+                newAST = reAnalyzeMaterializedViewAfterCbo(newAST);
+                // Store text of the ORIGINAL QUERY
+                String originalText = ctx.getTokenRewriteStream().toString(
+                    cboCtx.nodeOfInterest.getTokenStartIndex(),
+                    cboCtx.nodeOfInterest.getTokenStopIndex());
+                createVwDesc.setViewOriginalText(originalText);
+                viewSelect = newAST;
+                viewsExpanded = new ArrayList<>();
+                viewsExpanded.add(createVwDesc.getViewName());
+              } else {
+                // CTAS
+                setAST(newAST);
+                newAST = reAnalyzeCTASAfterCbo(newAST);
+              }
             }
             Phase1Ctx ctx_1 = initPhase1Ctx();
             if (!doPhase1(newAST, getQB(), ctx_1, null)) {
@@ -402,12 +418,13 @@ public class CalcitePlanner extends SemanticAnalyzer {
     int root = ast.getToken().getType();
     boolean needToLogMessage = STATIC_LOG.isInfoEnabled();
     boolean isSupportedRoot = root == HiveParser.TOK_QUERY || root == HiveParser.TOK_EXPLAIN
-        || qb.isCTAS();
+        || qb.isCTAS() || qb.isMaterializedView();
     // Queries without a source table currently are not supported by CBO
     boolean isSupportedType = (qb.getIsQuery() && !qb.containsQueryWithoutSourceTable())
-        || qb.isCTAS() || cboCtx.type == PreCboCtx.Type.INSERT;
+        || qb.isCTAS() || qb.isMaterializedView() || cboCtx.type == PreCboCtx.Type.INSERT;
     boolean noBadTokens = HiveCalciteUtil.validateASTForUnsupportedTokens(ast);
-    boolean result = isSupportedRoot && isSupportedType && getCreateViewDesc() == null
+    boolean result = isSupportedRoot && isSupportedType
+        && (getCreateViewDesc() == null || getCreateViewDesc().isMaterialized())
         && noBadTokens;
 
     if (!result) {
@@ -420,7 +437,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
           msg += "is not a query with at least one source table "
                   + " or there is a subquery without a source table, or CTAS, or insert; ";
         }
-        if (getCreateViewDesc() != null) {
+        if (getCreateViewDesc() != null && !getCreateViewDesc().isMaterialized()) {
           msg += "has create view; ";
         }
         if (!noBadTokens) {
@@ -592,7 +609,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
    */
   static class PreCboCtx extends PlannerContext {
     enum Type {
-      NONE, INSERT, CTAS, UNEXPECTED
+      NONE, INSERT, CTAS_OR_MV, UNEXPECTED
     }
 
     private ASTNode nodeOfInterest;
@@ -610,8 +627,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
     }
 
     @Override
-    void setCTASToken(ASTNode child) {
-      set(PreCboCtx.Type.CTAS, child);
+    void setCTASOrMVToken(ASTNode child) {
+      set(PreCboCtx.Type.CTAS_OR_MV, child);
     }
 
     @Override
@@ -622,7 +639,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
     }
   }
 
-  ASTNode fixUpCtasAndInsertAfterCbo(ASTNode originalAst, ASTNode newAst, PreCboCtx cboCtx)
+  ASTNode fixUpAfterCbo(ASTNode originalAst, ASTNode newAst, PreCboCtx cboCtx)
       throws SemanticException {
     switch (cboCtx.type) {
 
@@ -630,7 +647,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       // nothing to do
       return newAst;
 
-    case CTAS: {
+    case CTAS_OR_MV: {
       // Patch the optimized query back into original CTAS AST, replacing the
       // original query.
       replaceASTChild(cboCtx.nodeOfInterest, newAst);
@@ -655,7 +672,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
     }
   }
 
-  ASTNode reAnalyzeCtasAfterCbo(ASTNode newAst) throws SemanticException {
+  ASTNode reAnalyzeCTASAfterCbo(ASTNode newAst) throws SemanticException {
     // analyzeCreateTable uses this.ast, but doPhase1 doesn't, so only reset it
     // here.
     newAst = analyzeCreateTable(newAst, getQB(), null);
@@ -667,6 +684,18 @@ public class CalcitePlanner extends SemanticAnalyzer {
     return newAst;
   }
 
+  ASTNode reAnalyzeMaterializedViewAfterCbo(ASTNode newAst) throws SemanticException {
+    // analyzeCreateView uses this.ast, but doPhase1 doesn't, so only reset it
+    // here.
+    newAst = analyzeCreateView(newAst, getQB(), null);
+    if (newAst == null) {
+      LOG.error("analyzeCreateTable failed to initialize materialized view after CBO;" + " new ast is "
+          + getAST().dump());
+      throw new SemanticException("analyzeCreateTable failed to initialize materialized view after CBO");
+    }
+    return newAst;
+  }
+
   /**
    * Performs breadth-first search of the AST for a nested set of tokens. Tokens
    * don't have to be each others' direct children, they can be separated by

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
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 988c58e..8b0db4a 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
@@ -332,7 +332,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       break;
     }
     case HiveParser.TOK_DROPTABLE:
-      analyzeDropTable(ast, false);
+      analyzeDropTable(ast, null);
       break;
     case HiveParser.TOK_TRUNCATETABLE:
       analyzeTruncateTable(ast);
@@ -407,7 +407,10 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       analyzeMetastoreCheck(ast);
       break;
     case HiveParser.TOK_DROPVIEW:
-      analyzeDropTable(ast, true);
+      analyzeDropTable(ast, TableType.VIRTUAL_VIEW);
+      break;
+    case HiveParser.TOK_DROP_MATERIALIZED_VIEW:
+      analyzeDropTable(ast, TableType.MATERIALIZED_VIEW);
       break;
     case HiveParser.TOK_ALTERVIEW: {
       String[] qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
@@ -881,7 +884,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
 
 
-  private void analyzeDropTable(ASTNode ast, boolean expectView)
+  private void analyzeDropTable(ASTNode ast, TableType expectedType)
       throws SemanticException {
     String tableName = getUnescapedName((ASTNode) ast.getChild(0));
     boolean ifExists = (ast.getFirstChildWithType(HiveParser.TOK_IFEXISTS) != null);
@@ -899,7 +902,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     boolean ifPurge = (ast.getFirstChildWithType(HiveParser.KW_PURGE) != null);
-    DropTableDesc dropTblDesc = new DropTableDesc(tableName, expectView, ifExists, ifPurge, replicationSpec);
+    DropTableDesc dropTblDesc = new DropTableDesc(tableName, expectedType, ifExists, ifPurge, replicationSpec);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         dropTblDesc), conf));
   }
@@ -2736,7 +2739,8 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     addTableDropPartsOutputs(tab, partSpecs.values(), !ifExists);
 
     DropTableDesc dropTblDesc =
-        new DropTableDesc(getDotName(qualified), partSpecs, expectView, mustPurge, replicationSpec);
+        new DropTableDesc(getDotName(qualified), partSpecs, expectView ? TableType.VIRTUAL_VIEW : null,
+                mustPurge, replicationSpec);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropTblDesc), conf));
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 92dfc90..62bbcc6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -250,8 +250,8 @@ TOK_ALTERVIEW_DROPPROPERTIES;
 TOK_ALTERVIEW_ADDPARTS;
 TOK_ALTERVIEW_DROPPARTS;
 TOK_ALTERVIEW_RENAME;
-TOK_REBUILD_MATERIALIZED_VIEW;
 TOK_CREATE_MATERIALIZED_VIEW;
+TOK_DROP_MATERIALIZED_VIEW;
 TOK_VIEWPARTCOLS;
 TOK_EXPLAIN;
 TOK_EXPLAIN_SQ_REWRITE;
@@ -1034,7 +1034,6 @@ alterStatement
 @after { popMsg(state); }
     : KW_ALTER KW_TABLE tableName alterTableStatementSuffix -> ^(TOK_ALTERTABLE tableName alterTableStatementSuffix)
     | KW_ALTER KW_VIEW tableName KW_AS? alterViewStatementSuffix -> ^(TOK_ALTERVIEW tableName alterViewStatementSuffix)
-    | KW_ALTER KW_MATERIALIZED KW_VIEW tableName KW_REBUILD -> ^(TOK_REBUILD_MATERIALIZED_VIEW tableName)
     | KW_ALTER KW_INDEX alterIndexStatementSuffix -> alterIndexStatementSuffix
     | KW_ALTER (KW_DATABASE|KW_SCHEMA) alterDatabaseStatementSuffix -> alterDatabaseStatementSuffix
     ;
@@ -1789,11 +1788,14 @@ createMaterializedViewStatement
 }
 @after { popMsg(state); }
     : KW_CREATE KW_MATERIALIZED KW_VIEW (ifNotExists)? name=tableName
-        tableComment? tablePropertiesPrefixed?
-        KW_AS selectStatementWithCTE
+        tableComment? tableRowFormat? tableFileFormat? tableLocation?
+        tablePropertiesPrefixed? KW_AS selectStatementWithCTE
     -> ^(TOK_CREATE_MATERIALIZED_VIEW $name 
          ifNotExists?
          tableComment?
+         tableRowFormat?
+         tableFileFormat?
+         tableLocation?
          tablePropertiesPrefixed?
          selectStatementWithCTE
         )
@@ -1815,7 +1817,7 @@ dropViewStatement
 dropMaterializedViewStatement
 @init { pushMsg("drop materialized view statement", state); }
 @after { popMsg(state); }
-    : KW_DROP KW_MATERIALIZED KW_VIEW ifExists? viewName -> ^(TOK_DROPTABLE viewName ifExists?)
+    : KW_DROP KW_MATERIALIZED KW_VIEW ifExists? viewName -> ^(TOK_DROP_MATERIALIZED_VIEW viewName ifExists?)
     ;
 
 showFunctionIdentifier

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/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 d562ddf..9986fcf 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
@@ -345,7 +345,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
     return TaskFactory.get(new DDLWork(
         getInputs(),
         getOutputs(),
-        new DropTableDesc(table.getTableName(), false, true, true, null)
+        new DropTableDesc(table.getTableName(), null, true, true, null)
     ), conf);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index ab2966b..577d006 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -55,7 +55,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.hive.common.BlobStorageUtils;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.common.StatsSetupConst;
@@ -285,9 +284,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   Map<GroupByOperator, Set<String>> groupOpToInputTables;
   Map<String, PrunedPartitionList> prunedPartitions;
   protected List<FieldSchema> resultSchema;
-  private CreateViewDesc createVwDesc;
-  private ArrayList<String> viewsExpanded;
-  private ASTNode viewSelect;
+  protected CreateViewDesc createVwDesc;
+  protected ArrayList<String> viewsExpanded;
+  protected ASTNode viewSelect;
   protected final UnparseTranslator unparseTranslator;
   private final GlobalLimitCtx globalLimitCtx;
 
@@ -10514,7 +10513,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       this.ctx_1 = ctx_1;
     }
 
-    void setCTASToken(ASTNode child) {
+    void setCTASOrMVToken(ASTNode child) {
     }
 
     void setInsertToken(ASTNode ast, boolean isTmpFileDest) {
@@ -10699,10 +10698,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     // 3. analyze create view command
     if (ast.getToken().getType() == HiveParser.TOK_CREATEVIEW ||
         ast.getToken().getType() == HiveParser.TOK_CREATE_MATERIALIZED_VIEW ||
-        ast.getToken().getType() == HiveParser.TOK_REBUILD_MATERIALIZED_VIEW ||
         (ast.getToken().getType() == HiveParser.TOK_ALTERVIEW &&
             ast.getChild(1).getType() == HiveParser.TOK_QUERY)) {
-      child = analyzeCreateView(ast, qb);
+      child = analyzeCreateView(ast, qb, plannerCtx);
       if (child == null) {
         return false;
       }
@@ -11015,12 +11013,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   }
 
   private void saveViewDefinition() throws SemanticException {
-
-    if (createVwDesc.isMaterialized() && createVwDesc.getOrReplace()) {
-      // This is a rebuild, there's nothing to do here.
-      return;
-    }
-
     // Make a copy of the statement's result schema, since we may
     // modify it below as part of imposing view column names.
     List<FieldSchema> derivedSchema =
@@ -11039,9 +11031,11 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     // Preserve the original view definition as specified by the user.
-    String originalText = ctx.getTokenRewriteStream().toString(
-        viewSelect.getTokenStartIndex(), viewSelect.getTokenStopIndex());
-    createVwDesc.setViewOriginalText(originalText);
+    if (createVwDesc.getViewOriginalText() == null) {
+      String originalText = ctx.getTokenRewriteStream().toString(
+          viewSelect.getTokenStartIndex(), viewSelect.getTokenStopIndex());
+      createVwDesc.setViewOriginalText(originalText);
+    }
 
     // Now expand the view definition with extras such as explicit column
     // references; this expanded form is what we'll re-parse when the view is
@@ -11582,7 +11576,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         }
         command_type = CTAS;
         if (plannerCtx != null) {
-          plannerCtx.setCTASToken(child);
+          plannerCtx.setCTASOrMVToken(child);
         }
         selectStmt = child;
         break;
@@ -11820,7 +11814,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     outputs.add(new WriteEntity(t, WriteEntity.WriteType.DDL_NO_LOCK));
   }
 
-  private ASTNode analyzeCreateView(ASTNode ast, QB qb) throws SemanticException {
+  protected ASTNode analyzeCreateView(ASTNode ast, QB qb, PlannerContext plannerCtx) throws SemanticException {
     String[] qualTabName = getQualifiedTableName((ASTNode) ast.getChild(0));
     String dbDotTable = getDotName(qualTabName);
     List<FieldSchema> cols = null;
@@ -11831,17 +11825,19 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     ASTNode selectStmt = null;
     Map<String, String> tblProps = null;
     List<String> partColNames = null;
-    boolean isRebuild = ast.getToken().getType() == HiveParser.TOK_REBUILD_MATERIALIZED_VIEW;
-    boolean isMaterialized = ast.getToken().getType() == HiveParser.TOK_CREATE_MATERIALIZED_VIEW
-        || isRebuild;
-    StorageFormat storageFormat = storageFormat = new StorageFormat(conf);
-    storageFormat.fillDefaultStorageFormat(false, isMaterialized);
+    boolean isMaterialized = ast.getToken().getType() == HiveParser.TOK_CREATE_MATERIALIZED_VIEW;
+    String location = null;
+    RowFormatParams rowFormatParams = new RowFormatParams();
+    StorageFormat storageFormat = new StorageFormat(conf);
 
     LOG.info("Creating view " + dbDotTable + " position="
         + ast.getCharPositionInLine());
     int numCh = ast.getChildCount();
     for (int num = 1; num < numCh; num++) {
       ASTNode child = (ASTNode) ast.getChild(num);
+      if (storageFormat.fillStorageFormat(child)) {
+        continue;
+      }
       switch (child.getToken().getType()) {
       case HiveParser.TOK_IFNOTEXISTS:
         ifNotExists = true;
@@ -11850,6 +11846,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         orReplace = true;
         break;
       case HiveParser.TOK_QUERY:
+        // For CBO
+        if (plannerCtx != null) {
+          plannerCtx.setCTASOrMVToken(child);
+        }
         selectStmt = child;
         break;
       case HiveParser.TOK_TABCOLNAME:
@@ -11864,11 +11864,29 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       case HiveParser.TOK_VIEWPARTCOLS:
         partColNames = getColumnNames((ASTNode) child.getChild(0));
         break;
+      case HiveParser.TOK_TABLEROWFORMAT:
+        rowFormatParams.analyzeRowFormat(child);
+        break;
+      case HiveParser.TOK_TABLELOCATION:
+        location = unescapeSQLString(child.getChild(0).getText());
+        location = EximUtil.relativeToAbsolutePath(conf, location);
+        inputs.add(toReadEntity(location));
+        break;
+      case HiveParser.TOK_TABLESERIALIZER:
+        child = (ASTNode) child.getChild(0);
+        storageFormat.setSerde(unescapeSQLString(child.getChild(0).getText()));
+        if (child.getChildCount() == 2) {
+          readProps((ASTNode) (child.getChild(1).getChild(0)),
+              storageFormat.getSerdeProps());
+        }
+        break;
       default:
         assert false;
       }
     }
 
+    storageFormat.fillDefaultStorageFormat(false, isMaterialized);
+
     if (ifNotExists && orReplace){
       throw new SemanticException("Can't combine IF NOT EXISTS and OR REPLACE.");
     }
@@ -11882,9 +11900,11 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     unparseTranslator.enable();
 
     if (isMaterialized) {
-      createVwDesc = new CreateViewDesc(dbDotTable, cols, comment, tblProps, partColNames,
-              ifNotExists, orReplace || isRebuild, isAlterViewAs, storageFormat.getInputFormat(),
-              storageFormat.getOutputFormat(), storageFormat.getSerde());
+      createVwDesc = new CreateViewDesc(
+              dbDotTable, cols, comment, tblProps, partColNames,
+              ifNotExists, orReplace, isAlterViewAs, storageFormat.getInputFormat(),
+              storageFormat.getOutputFormat(), location, storageFormat.getSerde(),
+              storageFormat.getStorageHandler(), storageFormat.getSerdeProps());
       addDbAndTabToOutputs(qualTabName, TableType.MATERIALIZED_VIEW);
       queryState.setCommandType(HiveOperation.CREATE_MATERIALIZED_VIEW);
       qb.setViewDesc(createVwDesc);
@@ -11895,24 +11915,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
               storageFormat.getOutputFormat(), storageFormat.getSerde());
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
               createVwDesc), conf));
-
       addDbAndTabToOutputs(qualTabName, TableType.VIRTUAL_VIEW);
       queryState.setCommandType(HiveOperation.CREATEVIEW);
     }
 
-    if (isRebuild) {
-      // We need to go lookup the table and get the select statement and then parse it.
-      try {
-        Table tab = db.getTable(qualTabName[0], qualTabName[1]);
-        String viewText = tab.getViewOriginalText();
-        ParseDriver pd = new ParseDriver();
-        ASTNode tree = pd.parse(viewText, ctx, false);
-        selectStmt = ParseUtils.findRootNonNullToken(tree);
-      } catch (Exception e) {
-        throw new SemanticException(e);
-      }
-    }
-
     return selectStmt;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
index cc731bf..18439f9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
@@ -84,7 +84,9 @@ public final class SemanticAnalyzerFactory {
     commandType.put(HiveParser.TOK_CREATEMACRO, HiveOperation.CREATEMACRO);
     commandType.put(HiveParser.TOK_DROPMACRO, HiveOperation.DROPMACRO);
     commandType.put(HiveParser.TOK_CREATEVIEW, HiveOperation.CREATEVIEW);
+    commandType.put(HiveParser.TOK_CREATE_MATERIALIZED_VIEW, HiveOperation.CREATE_MATERIALIZED_VIEW);
     commandType.put(HiveParser.TOK_DROPVIEW, HiveOperation.DROPVIEW);
+    commandType.put(HiveParser.TOK_DROP_MATERIALIZED_VIEW, HiveOperation.DROP_MATERIALIZED_VIEW);
     commandType.put(HiveParser.TOK_CREATEINDEX, HiveOperation.CREATEINDEX);
     commandType.put(HiveParser.TOK_DROPINDEX, HiveOperation.DROPINDEX);
     commandType.put(HiveParser.TOK_ALTERINDEX_REBUILD, HiveOperation.ALTERINDEX_REBUILD);
@@ -231,6 +233,7 @@ public final class SemanticAnalyzerFactory {
       case HiveParser.TOK_SWITCHDATABASE:
       case HiveParser.TOK_DROPTABLE:
       case HiveParser.TOK_DROPVIEW:
+      case HiveParser.TOK_DROP_MATERIALIZED_VIEW:
       case HiveParser.TOK_DESCDATABASE:
       case HiveParser.TOK_DESCTABLE:
       case HiveParser.TOK_DESCFUNCTION:

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
index f781390..97cf585 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
@@ -231,10 +231,9 @@ public abstract class TaskCompiler {
           // make the movetask's destination directory the table's destination.
           Path location;
           String loc = pCtx.getQueryProperties().isCTAS() ?
-                  pCtx.getCreateTable().getLocation() : null;
+                  pCtx.getCreateTable().getLocation() : pCtx.getCreateViewDesc().getLocation();
           if (loc == null) {
-            // get the table's default location
-            // Always use default location for materialized view
+            // get the default location
             Path targetPath;
             try {
               String protoName = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
index 3ddbb1f..6830bda 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
@@ -36,8 +36,6 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
   private static final long serialVersionUID = 1L;
 
   private String viewName;
-  private String inputFormat;
-  private String outputFormat;
   private String originalText;
   private String expandedText;
   private List<FieldSchema> schema;
@@ -49,7 +47,12 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
   private boolean orReplace;
   private boolean isAlterViewAs;
   private boolean isMaterialized;
+  private String inputFormat;
+  private String outputFormat;
+  private String location; // only used for materialized views
   private String serde; // only used for materialized views
+  private String storageHandler; // only used for materialized views
+  private Map<String, String> serdeProps; // only used for materialized views
 
   /**
    * For serialization only.
@@ -58,7 +61,7 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
   }
 
   /**
-   * Used to create a virtual view descriptor.
+   * Used to create a materialized view descriptor
    * @param viewName
    * @param schema
    * @param comment
@@ -67,27 +70,37 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
    * @param ifNotExists
    * @param orReplace
    * @param isAlterViewAs
+   * @param inputFormat
+   * @param outputFormat
+   * @param location
+   * @param serde
+   * @param storageHandler
+   * @param serdeProps
    */
-  public CreateViewDesc(String viewName, List<FieldSchema> schema,
-      String comment, String inputFormat,
-      String outputFormat, Map<String, String> tblProps,
-      List<String> partColNames, boolean ifNotExists,
-      boolean orReplace, boolean isAlterViewAs) {
+  public CreateViewDesc(String viewName, List<FieldSchema> schema, String comment,
+          Map<String, String> tblProps, List<String> partColNames,
+          boolean ifNotExists, boolean orReplace, boolean isAlterViewAs,
+          String inputFormat, String outputFormat, String location,
+          String serde, String storageHandler, Map<String, String> serdeProps) {
     this.viewName = viewName;
     this.schema = schema;
-    this.comment = comment;
-    this.inputFormat = inputFormat;
-    this.outputFormat = outputFormat;
     this.tblProps = tblProps;
     this.partColNames = partColNames;
+    this.comment = comment;
     this.ifNotExists = ifNotExists;
     this.orReplace = orReplace;
     this.isAlterViewAs = isAlterViewAs;
-    this.isMaterialized = false;
+    this.isMaterialized = true;
+    this.inputFormat = inputFormat;
+    this.outputFormat = outputFormat;
+    this.location = location;
+    this.serde = serde;
+    this.storageHandler = storageHandler;
+    this.serdeProps = serdeProps;
   }
 
   /**
-   * Used to create a materialized view descriptor
+   * Used to create a view descriptor
    * @param viewName
    * @param schema
    * @param comment
@@ -112,7 +125,7 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
     this.ifNotExists = ifNotExists;
     this.orReplace = orReplace;
     this.isAlterViewAs = isAlterViewAs;
-    this.isMaterialized = true;
+    this.isMaterialized = false;
     this.inputFormat = inputFormat;
     this.outputFormat = outputFormat;
     this.serde = serde;
@@ -244,8 +257,20 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
     return isMaterialized;
   }
 
+  public String getLocation() {
+    return location;
+  }
+
   public String getSerde() {
     return serde;
   }
 
+  public String getStorageHandler() {
+    return storageHandler;
+  }
+
+  public Map<String, String> getSerdeProps() {
+    return serdeProps;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java
index 62c8f7e..74c742c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java
@@ -18,14 +18,15 @@
 
 package org.apache.hadoop.hive.ql.plan;
 
-import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
-import org.apache.hadoop.hive.ql.plan.Explain.Level;
-
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
+import org.apache.hadoop.hive.ql.plan.Explain.Level;
+
 /**
  * DropTableDesc.
  * TODO: this is currently used for both drop table and drop partitions.
@@ -53,10 +54,11 @@ public class DropTableDesc extends DDLDesc implements Serializable {
 
   String tableName;
   ArrayList<PartSpec> partSpecs;
-  boolean expectView;
+  TableType expectedType;
   boolean ifExists;
   boolean ifPurge;
   ReplicationSpec replicationSpec;
+  
 
   public DropTableDesc() {
   }
@@ -66,18 +68,18 @@ public class DropTableDesc extends DDLDesc implements Serializable {
    * @param ifPurge
    */
   public DropTableDesc(
-      String tableName, boolean expectView, boolean ifExists,
+      String tableName, TableType expectedType, boolean ifExists,
       boolean ifPurge, ReplicationSpec replicationSpec) {
     this.tableName = tableName;
     this.partSpecs = null;
-    this.expectView = expectView;
+    this.expectedType = expectedType;
     this.ifExists = ifExists;
     this.ifPurge = ifPurge;
     this.replicationSpec = replicationSpec;
   }
 
   public DropTableDesc(String tableName, Map<Integer, List<ExprNodeGenericFuncDesc>> partSpecs,
-      boolean expectView, boolean ifPurge, ReplicationSpec replicationSpec) {
+      TableType expectedType, boolean ifPurge, ReplicationSpec replicationSpec) {
     this.tableName = tableName;
     this.partSpecs = new ArrayList<PartSpec>(partSpecs.size());
     for (Map.Entry<Integer, List<ExprNodeGenericFuncDesc>> partSpec : partSpecs.entrySet()) {
@@ -86,7 +88,7 @@ public class DropTableDesc extends DDLDesc implements Serializable {
         this.partSpecs.add(new PartSpec(expr, prefixLength));
       }
     }
-    this.expectView = expectView;
+    this.expectedType = expectedType;
     this.ifPurge = ifPurge;
     this.replicationSpec = replicationSpec;
   }
@@ -118,15 +120,14 @@ public class DropTableDesc extends DDLDesc implements Serializable {
    * @return whether to expect a view being dropped
    */
   public boolean getExpectView() {
-    return expectView;
+    return expectedType != null && expectedType == TableType.VIRTUAL_VIEW;
   }
 
   /**
-   * @param expectView
-   *          set whether to expect a view being dropped
+   * @return whether to expect a materialized view being dropped
    */
-  public void setExpectView(boolean expectView) {
-    this.expectView = expectView;
+  public boolean getExpectMaterializedView() {
+    return expectedType != null && expectedType == TableType.MATERIALIZED_VIEW;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
index d5b1c9d..6fd4df4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
@@ -78,9 +78,10 @@ public enum HiveOperation {
   CREATEMACRO("CREATEMACRO", null, null),
   DROPMACRO("DROPMACRO", null, null),
   CREATEVIEW("CREATEVIEW", new Privilege[]{Privilege.SELECT}, new Privilege[]{Privilege.CREATE}),
-  CREATE_MATERIALIZED_VIEW("CREATEVIEW", new Privilege[]{Privilege.SELECT}, new
+  CREATE_MATERIALIZED_VIEW("CREATE_MATERIALIZED_VIEW", new Privilege[]{Privilege.SELECT}, new
       Privilege[]{Privilege.CREATE}),
   DROPVIEW("DROPVIEW", null, new Privilege[]{Privilege.DROP}),
+  DROP_MATERIALIZED_VIEW("DROP_MATERIALIZED_VIEW", null, new Privilege[]{Privilege.DROP}),
   CREATEINDEX("CREATEINDEX", null, null),
   DROPINDEX("DROPINDEX", null, null),
   ALTERINDEX_REBUILD("ALTERINDEX_REBUILD", null, null),

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
index 82e7fc5..4931e60 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
@@ -424,6 +424,8 @@ public class AuthorizationPreEventListener extends MetaStorePreEventListener {
           wrapperApiTable.setTableType(TableType.EXTERNAL_TABLE.toString());
         } else if (MetaStoreUtils.isIndexTable(wrapperApiTable)) {
           wrapperApiTable.setTableType(TableType.INDEX_TABLE.toString());
+        } else if (MetaStoreUtils.isMaterializedViewTable(wrapperApiTable)) {
+          wrapperApiTable.setTableType(TableType.MATERIALIZED_VIEW.toString());
         } else if ((wrapperApiTable.getSd() == null) || (wrapperApiTable.getSd().getLocation() == null)) {
           wrapperApiTable.setTableType(TableType.VIRTUAL_VIEW.toString());
         } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
index 884b129..2da4249 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
@@ -84,6 +84,7 @@ public enum HiveOperationType {
   CREATEVIEW,
   CREATE_MATERIALIZED_VIEW,
   DROPVIEW,
+  DROP_MATERIALIZED_VIEW,
   CREATEINDEX,
   DROPINDEX,
   ALTERINDEX_REBUILD,

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
index ddf1e66..e5389bd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
@@ -257,6 +257,8 @@ public class Operation2Privilege {
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.DROPVIEW, PrivRequirement.newIOPrivRequirement
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
+    op2Priv.put(HiveOperationType.DROP_MATERIALIZED_VIEW, PrivRequirement.newIOPrivRequirement
+(OWNER_PRIV_AR, OWNER_PRIV_AR));
 
     op2Priv.put(HiveOperationType.ANALYZE_TABLE, PrivRequirement.newIOPrivRequirement
 (arr(SQLPrivTypeGrant.SELECT_NOGRANT, SQLPrivTypeGrant.INSERT_NOGRANT), null));

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/queries/clientnegative/materialized_view_authorization_rebuild_no_grant.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/materialized_view_authorization_rebuild_no_grant.q b/ql/src/test/queries/clientnegative/materialized_view_authorization_rebuild_no_grant.q
deleted file mode 100644
index a2e7d38..0000000
--- a/ql/src/test/queries/clientnegative/materialized_view_authorization_rebuild_no_grant.q
+++ /dev/null
@@ -1,20 +0,0 @@
-set hive.test.authz.sstd.hs2.mode=true;
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
-set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
-set hive.security.authorization.enabled=true;
-set user.name=user1;
-
-create table amvrng_table (a int, b varchar(256), c decimal(10,2));
-
-insert into amvrng_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8);
-
-grant select on table amvrng_table to user user2 with grant option;
-
-set user.name=user2;
-create materialized view amvrng_mat_view as select a, c from amvrng_table;
-
-set user.name=user1;
-revoke grant option for select on table amvrng_table from user user2;
-
-set user.name=user2;
-alter materialized view amvrng_mat_view rebuild;

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/queries/clientnegative/materialized_view_authorization_rebuild_other.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/materialized_view_authorization_rebuild_other.q b/ql/src/test/queries/clientnegative/materialized_view_authorization_rebuild_other.q
deleted file mode 100644
index 7c2d145..0000000
--- a/ql/src/test/queries/clientnegative/materialized_view_authorization_rebuild_other.q
+++ /dev/null
@@ -1,14 +0,0 @@
-set hive.test.authz.sstd.hs2.mode=true;
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
-set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
-set hive.security.authorization.enabled=true;
-set user.name=user1;
-
-create table amvro_table (a int, b varchar(256), c decimal(10,2));
-
-insert into amvro_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8);
-
-create materialized view amvro_mat_view as select a, c from amvro_table;
-
-set user.name=user2;
-alter materialized view amvro_mat_view rebuild;

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/queries/clientnegative/materialized_view_drop.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/materialized_view_drop.q b/ql/src/test/queries/clientnegative/materialized_view_drop.q
new file mode 100644
index 0000000..f16d8a7
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/materialized_view_drop.q
@@ -0,0 +1,9 @@
+create table cmv_basetable (a int, b varchar(256), c decimal(10,2));
+
+insert into cmv_basetable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8);
+
+create materialized view cmv_mat_view as select a, b, c from cmv_basetable;
+
+drop materialized view cmv_mat_view;
+
+drop materialized view cmv_basetable;

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/queries/clientnegative/materialized_view_drop2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/materialized_view_drop2.q b/ql/src/test/queries/clientnegative/materialized_view_drop2.q
new file mode 100644
index 0000000..627a78b
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/materialized_view_drop2.q
@@ -0,0 +1,7 @@
+create table cmv_basetable (a int, b varchar(256), c decimal(10,2));
+
+insert into cmv_basetable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8);
+
+create materialized view cmv_mat_view as select a, b, c from cmv_basetable;
+
+drop view cmv_mat_view;

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/queries/clientnegative/view_delete.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/view_delete.q b/ql/src/test/queries/clientnegative/view_delete.q
index c9dc985..a19b0e3 100644
--- a/ql/src/test/queries/clientnegative/view_delete.q
+++ b/ql/src/test/queries/clientnegative/view_delete.q
@@ -4,7 +4,6 @@ set hive.enforce.bucketing=true;
 
 create table dv_basetable (a int, b varchar(256), c decimal(10,2));
 
-
 create view dv_view as select a, b, c from dv_basetable;
 
 delete from dv_view where b = 'fred';

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/queries/clientnegative/view_update.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/view_update.q b/ql/src/test/queries/clientnegative/view_update.q
index 3a54ccb..af27ae6 100644
--- a/ql/src/test/queries/clientnegative/view_update.q
+++ b/ql/src/test/queries/clientnegative/view_update.q
@@ -4,7 +4,6 @@ set hive.enforce.bucketing=true;
 
 create table uv_basetable (a int, b varchar(256), c decimal(10,2));
 
-
 create view uv_view as select a, b, c from uv_basetable;
 
 update uv_view set b = 'joe' where b = 'fred';

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/queries/clientpositive/materialized_view_authorization_sqlstd.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_authorization_sqlstd.q b/ql/src/test/queries/clientpositive/materialized_view_authorization_sqlstd.q
index 87a53e9..eef1b01 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_authorization_sqlstd.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_authorization_sqlstd.q
@@ -44,8 +44,6 @@ show grant user user4 on table amvs_mat_view;
 select * from amvs_mat_view;
 
 set user.name=user1;
-alter materialized view amvs_mat_view rebuild;
-
 grant select on table amvs_table to user user2 with grant option;
 set user.name=user2;
 create materialized view amvs_mat_view2 as select a, b from amvs_table;

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/queries/clientpositive/materialized_view_describe.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_describe.q b/ql/src/test/queries/clientpositive/materialized_view_describe.q
new file mode 100644
index 0000000..2c48e1d
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/materialized_view_describe.q
@@ -0,0 +1,58 @@
+create table cmv_basetable (a int, b varchar(256), c decimal(10,2));
+
+insert into cmv_basetable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8);
+
+create materialized view cmv_mat_view
+comment 'this is the first view'
+tblproperties ('key'='foo') as select a, c from cmv_basetable;
+
+describe cmv_mat_view;
+
+describe extended cmv_mat_view;
+
+describe formatted cmv_mat_view;
+
+show tblproperties cmv_mat_view;
+
+select a, c from cmv_mat_view;
+
+drop materialized view cmv_mat_view;
+
+create materialized view cmv_mat_view2
+comment 'this is the second view'
+stored as textfile
+tblproperties ('key'='alice','key2'='bob') as select a from cmv_basetable;
+
+describe formatted cmv_mat_view2;
+
+select a from cmv_mat_view2;
+
+drop materialized view cmv_mat_view2;
+
+create materialized view cmv_mat_view3
+comment 'this is the third view'
+row format
+  delimited fields terminated by '\t'
+as select * from cmv_basetable;
+
+describe formatted cmv_mat_view3;
+
+select a, b, c from cmv_mat_view3;
+
+select distinct a from cmv_mat_view3;
+
+drop materialized view cmv_mat_view3;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/t;
+
+create materialized view cmv_mat_view4
+comment 'this is the last view'
+stored as textfile
+location '${system:test.tmp.dir}/t'
+as select a from cmv_basetable;
+
+describe formatted cmv_mat_view4;
+
+select a from cmv_mat_view4;
+
+drop materialized view cmv_mat_view4;

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/queries/clientpositive/materialized_view_rebuild.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_rebuild.q b/ql/src/test/queries/clientpositive/materialized_view_rebuild.q
deleted file mode 100644
index c0a5530..0000000
--- a/ql/src/test/queries/clientpositive/materialized_view_rebuild.q
+++ /dev/null
@@ -1,13 +0,0 @@
-create table rmv_table (cint int, cstring1 string);
-
-insert into rmv_table values(1, 'fred'), (10, 'wilma');
-
-create materialized view rmv_mat_view as select cint, cstring1 from rmv_table where cint < 10;
-
-select * from rmv_mat_view;
-
-insert into rmv_table values(2, 'barney'), (11, 'betty');
-
-alter materialized view rmv_mat_view rebuild;
-
-select * from rmv_mat_view;

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientnegative/materialized_view_authorization_drop_other.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_authorization_drop_other.q.out b/ql/src/test/results/clientnegative/materialized_view_authorization_drop_other.q.out
index d1aff33..9be10d6 100644
--- a/ql/src/test/results/clientnegative/materialized_view_authorization_drop_other.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_authorization_drop_other.q.out
@@ -18,13 +18,13 @@ POSTHOOK: Lineage: amvdo_table.a EXPRESSION [(values__tmp__table__1)values__tmp_
 POSTHOOK: Lineage: amvdo_table.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
 POSTHOOK: Lineage: amvdo_table.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
 PREHOOK: query: create materialized view amvdo_mat_view as select a, c from amvdo_table
-PREHOOK: type: CREATEVIEW
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@amvdo_table
 PREHOOK: Output: database:default
 PREHOOK: Output: default@amvdo_mat_view
 POSTHOOK: query: create materialized view amvdo_mat_view as select a, c from amvdo_table
-POSTHOOK: type: CREATEVIEW
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
 POSTHOOK: Input: default@amvdo_table
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@amvdo_mat_view
-FAILED: HiveAccessControlException Permission denied: Principal [name=user2, type=USER] does not have following privileges for operation DROPTABLE [[OBJECT OWNERSHIP] on Object [type=TABLE_OR_VIEW, name=default.amvdo_mat_view]]
+FAILED: HiveAccessControlException Permission denied: Principal [name=user2, type=USER] does not have following privileges for operation DROP_MATERIALIZED_VIEW [[OBJECT OWNERSHIP] on Object [type=TABLE_OR_VIEW, name=default.amvdo_mat_view], [OBJECT OWNERSHIP] on Object [type=TABLE_OR_VIEW, name=default.amvdo_mat_view]]

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientnegative/materialized_view_authorization_no_select_perm.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_authorization_no_select_perm.q.out b/ql/src/test/results/clientnegative/materialized_view_authorization_no_select_perm.q.out
index a3a3405..57f992f 100644
--- a/ql/src/test/results/clientnegative/materialized_view_authorization_no_select_perm.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_authorization_no_select_perm.q.out
@@ -18,12 +18,12 @@ POSTHOOK: Lineage: amvnsp_table.a EXPRESSION [(values__tmp__table__1)values__tmp
 POSTHOOK: Lineage: amvnsp_table.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
 POSTHOOK: Lineage: amvnsp_table.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
 PREHOOK: query: create materialized view amvnsp_mat_view as select a, c from amvnsp_table
-PREHOOK: type: CREATEVIEW
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@amvnsp_table
 PREHOOK: Output: database:default
 PREHOOK: Output: default@amvnsp_mat_view
 POSTHOOK: query: create materialized view amvnsp_mat_view as select a, c from amvnsp_table
-POSTHOOK: type: CREATEVIEW
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
 POSTHOOK: Input: default@amvnsp_table
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@amvnsp_mat_view

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_no_grant.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_no_grant.q.out b/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_no_grant.q.out
deleted file mode 100644
index dc7a58d..0000000
--- a/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_no_grant.q.out
+++ /dev/null
@@ -1,42 +0,0 @@
-PREHOOK: query: create table amvrng_table (a int, b varchar(256), c decimal(10,2))
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@amvrng_table
-POSTHOOK: query: create table amvrng_table (a int, b varchar(256), c decimal(10,2))
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@amvrng_table
-PREHOOK: query: insert into amvrng_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
-PREHOOK: type: QUERY
-PREHOOK: Input: default@values__tmp__table__1
-PREHOOK: Output: default@amvrng_table
-POSTHOOK: query: insert into amvrng_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@values__tmp__table__1
-POSTHOOK: Output: default@amvrng_table
-POSTHOOK: Lineage: amvrng_table.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: amvrng_table.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
-POSTHOOK: Lineage: amvrng_table.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
-PREHOOK: query: grant select on table amvrng_table to user user2 with grant option
-PREHOOK: type: GRANT_PRIVILEGE
-PREHOOK: Output: default@amvrng_table
-POSTHOOK: query: grant select on table amvrng_table to user user2 with grant option
-POSTHOOK: type: GRANT_PRIVILEGE
-POSTHOOK: Output: default@amvrng_table
-PREHOOK: query: create materialized view amvrng_mat_view as select a, c from amvrng_table
-PREHOOK: type: CREATEVIEW
-PREHOOK: Input: default@amvrng_table
-PREHOOK: Output: database:default
-PREHOOK: Output: default@amvrng_mat_view
-POSTHOOK: query: create materialized view amvrng_mat_view as select a, c from amvrng_table
-POSTHOOK: type: CREATEVIEW
-POSTHOOK: Input: default@amvrng_table
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@amvrng_mat_view
-PREHOOK: query: revoke grant option for select on table amvrng_table from user user2
-PREHOOK: type: REVOKE_PRIVILEGE
-PREHOOK: Output: default@amvrng_table
-POSTHOOK: query: revoke grant option for select on table amvrng_table from user user2
-POSTHOOK: type: REVOKE_PRIVILEGE
-POSTHOOK: Output: default@amvrng_table
-FAILED: HiveAccessControlException Permission denied: Principal [name=user2, type=USER] does not have following privileges for operation CREATE_MATERIALIZED_VIEW [[SELECT with grant] on Object [type=TABLE_OR_VIEW, name=default.amvrng_table]]

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_other.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_other.q.out b/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_other.q.out
deleted file mode 100644
index 9c25b52..0000000
--- a/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_other.q.out
+++ /dev/null
@@ -1,30 +0,0 @@
-PREHOOK: query: create table amvro_table (a int, b varchar(256), c decimal(10,2))
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@amvro_table
-POSTHOOK: query: create table amvro_table (a int, b varchar(256), c decimal(10,2))
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@amvro_table
-PREHOOK: query: insert into amvro_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
-PREHOOK: type: QUERY
-PREHOOK: Input: default@values__tmp__table__1
-PREHOOK: Output: default@amvro_table
-POSTHOOK: query: insert into amvro_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@values__tmp__table__1
-POSTHOOK: Output: default@amvro_table
-POSTHOOK: Lineage: amvro_table.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: amvro_table.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
-POSTHOOK: Lineage: amvro_table.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
-PREHOOK: query: create materialized view amvro_mat_view as select a, c from amvro_table
-PREHOOK: type: CREATEVIEW
-PREHOOK: Input: default@amvro_table
-PREHOOK: Output: database:default
-PREHOOK: Output: default@amvro_mat_view
-POSTHOOK: query: create materialized view amvro_mat_view as select a, c from amvro_table
-POSTHOOK: type: CREATEVIEW
-POSTHOOK: Input: default@amvro_table
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@amvro_mat_view
-FAILED: HiveAccessControlException Permission denied: Principal [name=user2, type=USER] does not have following privileges for operation CREATE_MATERIALIZED_VIEW [[SELECT with grant] on Object [type=TABLE_OR_VIEW, name=default.amvro_table]]

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientnegative/materialized_view_delete.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_delete.q.out b/ql/src/test/results/clientnegative/materialized_view_delete.q.out
index 6013e1c..c4a5887 100644
--- a/ql/src/test/results/clientnegative/materialized_view_delete.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_delete.q.out
@@ -7,13 +7,13 @@ POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@dmv_basetable
 PREHOOK: query: create materialized view dmv_mat_view as select a, b, c from dmv_basetable
-PREHOOK: type: CREATEVIEW
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@dmv_basetable
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dmv_mat_view
 POSTHOOK: query: create materialized view dmv_mat_view as select a, b, c from dmv_basetable
-POSTHOOK: type: CREATEVIEW
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
 POSTHOOK: Input: default@dmv_basetable
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@dmv_mat_view
-FAILED: SemanticException [Error 10304]: You cannot update or delete records in a view
+FAILED: SemanticException [Error 10402]: You cannot update or delete records in a view

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientnegative/materialized_view_drop.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_drop.q.out b/ql/src/test/results/clientnegative/materialized_view_drop.q.out
new file mode 100644
index 0000000..f3ac66c
--- /dev/null
+++ b/ql/src/test/results/clientnegative/materialized_view_drop.q.out
@@ -0,0 +1,42 @@
+PREHOOK: query: create table cmv_basetable (a int, b varchar(256), c decimal(10,2))
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@cmv_basetable
+POSTHOOK: query: create table cmv_basetable (a int, b varchar(256), c decimal(10,2))
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@cmv_basetable
+PREHOOK: query: insert into cmv_basetable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@cmv_basetable
+POSTHOOK: query: insert into cmv_basetable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@cmv_basetable
+POSTHOOK: Lineage: cmv_basetable.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: cmv_basetable.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: cmv_basetable.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+PREHOOK: query: create materialized view cmv_mat_view as select a, b, c from cmv_basetable
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
+PREHOOK: Input: default@cmv_basetable
+PREHOOK: Output: database:default
+PREHOOK: Output: default@cmv_mat_view
+POSTHOOK: query: create materialized view cmv_mat_view as select a, b, c from cmv_basetable
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
+POSTHOOK: Input: default@cmv_basetable
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@cmv_mat_view
+PREHOOK: query: drop materialized view cmv_mat_view
+PREHOOK: type: DROP_MATERIALIZED_VIEW
+PREHOOK: Input: default@cmv_mat_view
+PREHOOK: Output: default@cmv_mat_view
+POSTHOOK: query: drop materialized view cmv_mat_view
+POSTHOOK: type: DROP_MATERIALIZED_VIEW
+POSTHOOK: Input: default@cmv_mat_view
+POSTHOOK: Output: default@cmv_mat_view
+PREHOOK: query: drop materialized view cmv_basetable
+PREHOOK: type: DROP_MATERIALIZED_VIEW
+PREHOOK: Input: default@cmv_basetable
+PREHOOK: Output: default@cmv_basetable
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Cannot drop a base table with DROP MATERIALIZED VIEW

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientnegative/materialized_view_drop2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_drop2.q.out b/ql/src/test/results/clientnegative/materialized_view_drop2.q.out
new file mode 100644
index 0000000..46a9e1f
--- /dev/null
+++ b/ql/src/test/results/clientnegative/materialized_view_drop2.q.out
@@ -0,0 +1,34 @@
+PREHOOK: query: create table cmv_basetable (a int, b varchar(256), c decimal(10,2))
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@cmv_basetable
+POSTHOOK: query: create table cmv_basetable (a int, b varchar(256), c decimal(10,2))
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@cmv_basetable
+PREHOOK: query: insert into cmv_basetable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@cmv_basetable
+POSTHOOK: query: insert into cmv_basetable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@cmv_basetable
+POSTHOOK: Lineage: cmv_basetable.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: cmv_basetable.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: cmv_basetable.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+PREHOOK: query: create materialized view cmv_mat_view as select a, b, c from cmv_basetable
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
+PREHOOK: Input: default@cmv_basetable
+PREHOOK: Output: database:default
+PREHOOK: Output: default@cmv_mat_view
+POSTHOOK: query: create materialized view cmv_mat_view as select a, b, c from cmv_basetable
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
+POSTHOOK: Input: default@cmv_basetable
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@cmv_mat_view
+PREHOOK: query: drop view cmv_mat_view
+PREHOOK: type: DROPVIEW
+PREHOOK: Input: default@cmv_mat_view
+PREHOOK: Output: default@cmv_mat_view
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Cannot drop a materialized view with DROP VIEW

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientnegative/materialized_view_insert.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_insert.q.out b/ql/src/test/results/clientnegative/materialized_view_insert.q.out
index 08b9a78..3e1f591 100644
--- a/ql/src/test/results/clientnegative/materialized_view_insert.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_insert.q.out
@@ -7,12 +7,12 @@ POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@imv_basetable
 PREHOOK: query: create materialized view imv_mat_view as select a, b, c from imv_basetable
-PREHOOK: type: CREATEVIEW
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@imv_basetable
 PREHOOK: Output: database:default
 PREHOOK: Output: default@imv_mat_view
 POSTHOOK: query: create materialized view imv_mat_view as select a, b, c from imv_basetable
-POSTHOOK: type: CREATEVIEW
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
 POSTHOOK: Input: default@imv_basetable
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@imv_mat_view

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientnegative/materialized_view_load.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_load.q.out b/ql/src/test/results/clientnegative/materialized_view_load.q.out
index 4814365..b48bba4 100644
--- a/ql/src/test/results/clientnegative/materialized_view_load.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_load.q.out
@@ -7,12 +7,12 @@ POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@lmv_basetable
 PREHOOK: query: create materialized view lmv_mat_view as select a, b, c from lmv_basetable
-PREHOOK: type: CREATEVIEW
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@lmv_basetable
 PREHOOK: Output: database:default
 PREHOOK: Output: default@lmv_mat_view
 POSTHOOK: query: create materialized view lmv_mat_view as select a, b, c from lmv_basetable
-POSTHOOK: type: CREATEVIEW
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
 POSTHOOK: Input: default@lmv_basetable
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@lmv_mat_view

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientnegative/materialized_view_replace_with_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_replace_with_view.q.out b/ql/src/test/results/clientnegative/materialized_view_replace_with_view.q.out
index dac3538..45160ad 100644
--- a/ql/src/test/results/clientnegative/materialized_view_replace_with_view.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_replace_with_view.q.out
@@ -18,13 +18,13 @@ POSTHOOK: Lineage: rmvwv_basetable.a EXPRESSION [(values__tmp__table__1)values__
 POSTHOOK: Lineage: rmvwv_basetable.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
 POSTHOOK: Lineage: rmvwv_basetable.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
 PREHOOK: query: create materialized view rmvwv_mat_view as select a, b, c from rmvwv_basetable
-PREHOOK: type: CREATEVIEW
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@rmvwv_basetable
 PREHOOK: Output: database:default
 PREHOOK: Output: default@rmvwv_mat_view
 POSTHOOK: query: create materialized view rmvwv_mat_view as select a, b, c from rmvwv_basetable
-POSTHOOK: type: CREATEVIEW
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
 POSTHOOK: Input: default@rmvwv_basetable
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@rmvwv_mat_view
-FAILED: SemanticException [Error 10303]: Attempt to replace materialized view rmvwv_mat_view with view
+FAILED: SemanticException [Error 10401]: Attempt to replace materialized view rmvwv_mat_view with view

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientnegative/materialized_view_update.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_update.q.out b/ql/src/test/results/clientnegative/materialized_view_update.q.out
index bee63a8..b9e7761 100644
--- a/ql/src/test/results/clientnegative/materialized_view_update.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_update.q.out
@@ -7,13 +7,13 @@ POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@umv_basetable
 PREHOOK: query: create materialized view umv_mat_view as select a, b, c from umv_basetable
-PREHOOK: type: CREATEVIEW
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@umv_basetable
 PREHOOK: Output: database:default
 PREHOOK: Output: default@umv_mat_view
 POSTHOOK: query: create materialized view umv_mat_view as select a, b, c from umv_basetable
-POSTHOOK: type: CREATEVIEW
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
 POSTHOOK: Input: default@umv_basetable
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@umv_mat_view
-FAILED: SemanticException [Error 10304]: You cannot update or delete records in a view
+FAILED: SemanticException [Error 10402]: You cannot update or delete records in a view

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientnegative/view_delete.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/view_delete.q.out b/ql/src/test/results/clientnegative/view_delete.q.out
index 3e72344..24e7ba5 100644
--- a/ql/src/test/results/clientnegative/view_delete.q.out
+++ b/ql/src/test/results/clientnegative/view_delete.q.out
@@ -16,4 +16,4 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@dv_basetable
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@dv_view
-FAILED: SemanticException [Error 10304]: You cannot update or delete records in a view
+FAILED: SemanticException [Error 10402]: You cannot update or delete records in a view

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientnegative/view_update.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/view_update.q.out b/ql/src/test/results/clientnegative/view_update.q.out
index 8a12665..aaee442 100644
--- a/ql/src/test/results/clientnegative/view_update.q.out
+++ b/ql/src/test/results/clientnegative/view_update.q.out
@@ -16,4 +16,4 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@uv_basetable
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@uv_view
-FAILED: SemanticException [Error 10304]: You cannot update or delete records in a view
+FAILED: SemanticException [Error 10402]: You cannot update or delete records in a view

http://git-wip-us.apache.org/repos/asf/hive/blob/5179558c/ql/src/test/results/clientpositive/materialized_view_authorization_sqlstd.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/materialized_view_authorization_sqlstd.q.out b/ql/src/test/results/clientpositive/materialized_view_authorization_sqlstd.q.out
index b167702..25f081b 100644
--- a/ql/src/test/results/clientpositive/materialized_view_authorization_sqlstd.q.out
+++ b/ql/src/test/results/clientpositive/materialized_view_authorization_sqlstd.q.out
@@ -18,12 +18,12 @@ POSTHOOK: Lineage: amvs_table.a EXPRESSION [(values__tmp__table__1)values__tmp__
 POSTHOOK: Lineage: amvs_table.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
 POSTHOOK: Lineage: amvs_table.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
 PREHOOK: query: create materialized view amvs_mat_view as select a, c from amvs_table
-PREHOOK: type: CREATEVIEW
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@amvs_table
 PREHOOK: Output: database:default
 PREHOOK: Output: default@amvs_mat_view
 POSTHOOK: query: create materialized view amvs_mat_view as select a, c from amvs_table
-POSTHOOK: type: CREATEVIEW
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
 POSTHOOK: Input: default@amvs_table
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@amvs_mat_view
@@ -54,11 +54,11 @@ POSTHOOK: query: select * from amvs_mat_view
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@amvs_mat_view
 #### A masked pattern was here ####
-1	10.3
+1	10.30
 2	3.14
-2	172342.2
+2	172342.20
 3	978.76
-3	9.8
+3	9.80
 PREHOOK: query: show grant user user3 on table amvs_mat_view
 PREHOOK: type: SHOW_GRANT
 POSTHOOK: query: show grant user user3 on table amvs_mat_view
@@ -137,21 +137,11 @@ POSTHOOK: query: select * from amvs_mat_view
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@amvs_mat_view
 #### A masked pattern was here ####
-1	10.3
+1	10.30
 2	3.14
-2	172342.2
+2	172342.20
 3	978.76
-3	9.8
-PREHOOK: query: alter materialized view amvs_mat_view rebuild
-PREHOOK: type: CREATEVIEW
-PREHOOK: Input: default@amvs_table
-PREHOOK: Output: database:default
-PREHOOK: Output: default@amvs_mat_view
-POSTHOOK: query: alter materialized view amvs_mat_view rebuild
-POSTHOOK: type: CREATEVIEW
-POSTHOOK: Input: default@amvs_table
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@amvs_mat_view
+3	9.80
 PREHOOK: query: grant select on table amvs_table to user user2 with grant option
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@amvs_table
@@ -159,12 +149,12 @@ POSTHOOK: query: grant select on table amvs_table to user user2 with grant optio
 POSTHOOK: type: GRANT_PRIVILEGE
 POSTHOOK: Output: default@amvs_table
 PREHOOK: query: create materialized view amvs_mat_view2 as select a, b from amvs_table
-PREHOOK: type: CREATEVIEW
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@amvs_table
 PREHOOK: Output: database:default
 PREHOOK: Output: default@amvs_mat_view2
 POSTHOOK: query: create materialized view amvs_mat_view2 as select a, b from amvs_table
-POSTHOOK: type: CREATEVIEW
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
 POSTHOOK: Input: default@amvs_table
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@amvs_mat_view2
@@ -182,11 +172,11 @@ POSTHOOK: Input: default@amvs_mat_view2
 3	calvin
 3	charlie
 PREHOOK: query: drop materialized view amvs_mat_view2
-PREHOOK: type: DROPTABLE
+PREHOOK: type: DROP_MATERIALIZED_VIEW
 PREHOOK: Input: default@amvs_mat_view2
 PREHOOK: Output: default@amvs_mat_view2
 POSTHOOK: query: drop materialized view amvs_mat_view2
-POSTHOOK: type: DROPTABLE
+POSTHOOK: type: DROP_MATERIALIZED_VIEW
 POSTHOOK: Input: default@amvs_mat_view2
 POSTHOOK: Output: default@amvs_mat_view2
 PREHOOK: query: set role ADMIN