You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mg...@apache.org on 2020/07/15 13:13:46 UTC

[hive] branch master updated: HIVE-23244 Extract Create View analyzer from SemanticAnalyzer (Miklos Gergely, reviewed by Jesus Camacho Rodriguez) (#1125)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 2731daf  HIVE-23244 Extract Create View analyzer from SemanticAnalyzer (Miklos Gergely, reviewed by Jesus Camacho Rodriguez) (#1125)
2731daf is described below

commit 2731dafd89d57f1fc747779fb98fbe82a3e7c1bf
Author: Miklos Gergely <mg...@cloudera.com>
AuthorDate: Wed Jul 15 15:13:34 2020 +0200

    HIVE-23244 Extract Create View analyzer from SemanticAnalyzer (Miklos Gergely, reviewed by Jesus Camacho Rodriguez) (#1125)
---
 .../java/org/apache/hadoop/hive/ql/ErrorMsg.java   |   1 +
 .../org/apache/hadoop/hive/ql/parse/HiveParser.g   |   3 +-
 .../org/apache/hadoop/hive/ql/ddl/DDLUtils.java    |  14 +
 .../view/create/AbstractCreateViewAnalyzer.java    | 130 +++++++
 .../ql/ddl/view/create/AbstractCreateViewDesc.java |  74 ++++
 .../ql/ddl/view/create/AlterViewAsAnalyzer.java    |  92 +++++
 .../hive/ql/ddl/view/create/AlterViewAsDesc.java   |  37 ++
 .../ql/ddl/view/create/AlterViewAsOperation.java   |  52 +++
 ...ewDesc.java => CreateMaterializedViewDesc.java} | 182 +++------
 ...n.java => CreateMaterializedViewOperation.java} |  54 +--
 .../ql/ddl/view/create/CreateViewAnalyzer.java     | 214 +++++++++++
 .../hive/ql/ddl/view/create/CreateViewDesc.java    | 425 ++-------------------
 .../ql/ddl/view/create/CreateViewOperation.java    |  97 +++--
 .../hadoop/hive/ql/exec/repl/ReplLoadTask.java     |  14 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java    |   4 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java       |   8 +-
 .../apache/hadoop/hive/ql/parse/ParseContext.java  |   8 +-
 .../java/org/apache/hadoop/hive/ql/parse/QB.java   |  16 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java     | 417 +++++++-------------
 .../apache/hadoop/hive/ql/parse/StorageFormat.java |   2 +-
 .../apache/hadoop/hive/ql/parse/TaskCompiler.java  |  24 +-
 .../apache/hadoop/hive/ql/plan/HiveOperation.java  |   4 +-
 .../apache/hadoop/hive/ql/plan/LoadFileDesc.java   |  10 +-
 .../org/apache/hadoop/hive/ql/plan/PlanUtils.java  |   6 +-
 .../clientnegative/create_or_replace_view4.q.out   |   2 +-
 .../clientnegative/create_view_failure10.q.out     |   2 +-
 .../clientnegative/create_view_failure3.q.out      |   2 +-
 .../clientnegative/create_view_failure5.q.out      |   2 +-
 .../clientnegative/create_view_failure6.q.out      |   2 +-
 .../clientnegative/create_view_failure7.q.out      |   2 +-
 .../clientnegative/create_view_failure8.q.out      |   2 +-
 .../clientnegative/create_view_failure9.q.out      |   2 +-
 .../test/results/clientnegative/masking_mv.q.out   |   2 +-
 .../clientnegative/selectDistinctStarNeg_1.q.out   |   2 +-
 .../results/clientpositive/llap/create_view.q.out  |   8 +-
 .../llap/create_view_translate.q.out               |   8 +-
 .../results/clientpositive/llap/explain_ddl.q.out  |   8 +-
 .../clientpositive/llap/explainuser_1.q.out        |   4 +-
 .../results/clientpositive/llap/lineage3.q.out     |   6 +-
 .../results/clientpositive/llap/masking_mv.q.out   |   4 +-
 .../llap/materialized_view_cluster.q.out           |   2 +-
 .../llap/materialized_view_create_rewrite_3.q.out  |   2 +-
 .../llap/materialized_view_create_rewrite_4.q.out  |   2 +-
 ...ialized_view_create_rewrite_rebuild_dummy.q.out |   2 +-
 ...erialized_view_create_rewrite_time_window.q.out |   2 +-
 .../llap/materialized_view_distribute_sort.q.out   |   6 +-
 .../llap/materialized_view_partition_cluster.q.out |   2 +-
 .../llap/materialized_view_partitioned.q.out       |   2 +-
 .../llap/materialized_view_partitioned_3.q.out     |   2 +-
 .../clientpositive/llap/selectDistinctStar.q.out   |  16 +-
 .../llap/sketches_materialized_view_safety.q.out   |   2 +-
 .../clientpositive/llap/union_top_level.q.out      |   8 +-
 .../clientpositive/llap/vector_windowing.q.out     |  20 +-
 .../clientpositive/tez/explainanalyze_3.q.out      |   4 +-
 .../results/clientpositive/tez/explainuser_3.q.out |   4 +-
 55 files changed, 1001 insertions(+), 1021 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
index d732004..4b63653 100644
--- a/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
+++ b/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
@@ -439,6 +439,7 @@ public enum ErrorMsg {
   LOCK_ACQUIRE_CANCELLED(10330, "Query was cancelled while acquiring locks on the underlying objects. "),
   NOT_RECOGNIZED_CONSTRAINT(10331, "Constraint not recognized"),
   INVALID_CONSTRAINT(10332, "Invalid constraint definition"),
+  @Deprecated // kept for backwards reference
   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"),
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index dc51dd4..594d8c6 100644
--- a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -269,6 +269,7 @@ TOK_ALTERVIEW_DROPPROPERTIES;
 TOK_ALTERVIEW_ADDPARTS;
 TOK_ALTERVIEW_DROPPARTS;
 TOK_ALTERVIEW_RENAME;
+TOK_ALTERVIEW_AS;
 TOK_CREATE_MATERIALIZED_VIEW;
 TOK_DROP_MATERIALIZED_VIEW;
 TOK_ALTER_MATERIALIZED_VIEW;
@@ -1259,7 +1260,7 @@ alterViewStatementSuffix
     | alterStatementSuffixRename[false]
     | alterStatementSuffixAddPartitions[false]
     | alterStatementSuffixDropPartitions[false]
-    | selectStatementWithCTE
+    | selectStatementWithCTE -> ^(TOK_ALTERVIEW_AS selectStatementWithCTE)
     ;
 
 alterMaterializedViewStatementSuffix[CommonTree tableNameTree]
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLUtils.java
index b82fc5e..0aea302 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLUtils.java
@@ -32,8 +32,11 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.hooks.Entity.Type;
@@ -237,4 +240,15 @@ public final class DDLUtils {
       return null;
     }
   }
+
+  public static void addDbAndTableToOutputs(Database database, TableName tableName, TableType type, boolean isTemporary,
+      Map<String, String> properties, Set<WriteEntity> outputs) {
+    outputs.add(new WriteEntity(database, WriteEntity.WriteType.DDL_SHARED));
+
+    Table table = new Table(tableName.getDb(), tableName.getTable());
+    table.setParameters(properties);
+    table.setTableType(type);
+    table.setTemporary(isTemporary);
+    outputs.add(new WriteEntity(table, WriteEntity.WriteType.DDL_NO_LOCK));
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AbstractCreateViewAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AbstractCreateViewAnalyzer.java
new file mode 100644
index 0000000..1f9f103
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AbstractCreateViewAnalyzer.java
@@ -0,0 +1,130 @@
+/*
+ * 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.create;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.PlanUtils;
+
+/**
+ * Abstract ancestor of analyzers that can create a view.
+ */
+abstract class AbstractCreateViewAnalyzer extends BaseSemanticAnalyzer {
+  AbstractCreateViewAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  protected List<FieldSchema> schema;
+
+  @Override
+  public List<FieldSchema> getResultSchema() {
+    return schema;
+  }
+
+  protected SemanticAnalyzer analyzeQuery(ASTNode select, String fqViewName) throws SemanticException {
+    QueryState innerQueryState = new QueryState.Builder().withHiveConf(conf).build();
+    innerQueryState.getConf().setBoolVar(HiveConf.ConfVars.HIVE_RESULTSET_USE_UNIQUE_COLUMN_NAMES, false);
+
+    SemanticAnalyzer analyzer = (SemanticAnalyzer) SemanticAnalyzerFactory.get(innerQueryState, select);
+    ctx.setEnableUnparse(true);
+    analyzer.forViewCreation(fqViewName);
+    analyzer.analyze(select, ctx);
+    analyzer.executeUnparseTranlations();
+
+    queryState.setLineageState(innerQueryState.getLineageState());
+    queryState.getLineageState().mapDirToOp(new Path(fqViewName), analyzer.getSinkOp());
+
+    addInputs(analyzer);
+
+    return analyzer;
+  }
+
+  private void addInputs(SemanticAnalyzer analyzer) {
+    inputs.addAll(analyzer.getInputs());
+    for (Map.Entry<String, TableScanOperator> entry : analyzer.getTopOps().entrySet()) {
+      String alias = entry.getKey();
+      TableScanOperator topOp = entry.getValue();
+      ReadEntity parentViewInfo = PlanUtils.getParentViewInfo(alias, analyzer.getViewAliasToInput());
+
+      // Adds tables only for create view (PPD filter can be appended by outer query)
+      Table table = topOp.getConf().getTableMetadata();
+      PlanUtils.addInput(inputs, new ReadEntity(table, parentViewInfo));
+    }
+  }
+
+  protected void validateTablesUsed(SemanticAnalyzer analyzer) throws SemanticException {
+    // Do not allow view to be defined on temp table or other materialized view
+    Set<String> tableAliases = analyzer.getQB().getTabAliases();
+    for (String alias : tableAliases) {
+      if (SemanticAnalyzer.DUMMY_TABLE.equals(alias)) {
+        continue;
+      }
+      Table table = null;
+      try {
+        table = analyzer.getTableObjectByName(analyzer.getQB().getTabNameForAlias(alias));
+      } catch (HiveException ex) {
+        throw new SemanticException(ex);
+      }
+
+      if (table.isTemporary()) {
+        throw new SemanticException("View definition references temporary table " + alias);
+      }
+
+      if (table.isMaterializedView()) {
+        throw new SemanticException("View definition references materialized view " + alias);
+      }
+    }
+  }
+
+  protected void validateReplaceWithPartitions(String viewName, Table oldView, List<FieldSchema> partitionColumns)
+      throws SemanticException {
+    if (oldView.getPartCols().isEmpty() || oldView.getPartCols().equals(partitionColumns)) {
+      return;
+    }
+
+    String partitionViewErrorMsg = "The following view has partition, it could not be replaced: " + viewName;
+    List<Partition> partitions = null;
+    try {
+      partitions = db.getPartitions(oldView);
+    } catch (HiveException e) {
+      throw new SemanticException(ErrorMsg.REPLACE_VIEW_WITH_PARTITION.getMsg(partitionViewErrorMsg));
+    }
+
+    if (!partitions.isEmpty()) {
+      throw new SemanticException(ErrorMsg.REPLACE_VIEW_WITH_PARTITION.getMsg(partitionViewErrorMsg));
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AbstractCreateViewDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AbstractCreateViewDesc.java
new file mode 100644
index 0000000..03b5696
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AbstractCreateViewDesc.java
@@ -0,0 +1,74 @@
+/*
+ * 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.create;
+
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.ddl.DDLDesc;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.plan.Explain;
+import org.apache.hadoop.hive.ql.plan.Explain.Level;
+
+/**
+ * Abstract ancestor of view creating DDL commands.
+ */
+abstract class AbstractCreateViewDesc implements DDLDesc, Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final String viewName;
+  private final List<FieldSchema> schema;
+  private final String originalText;
+  private final String expandedText;
+
+  /**
+   * Used to create a view descriptor.
+   */
+  AbstractCreateViewDesc(String viewName, List<FieldSchema> schema, String originalText, String expandedText) {
+    this.viewName = viewName;
+    this.schema = schema;
+    this.originalText = originalText;
+    this.expandedText = expandedText;
+  }
+
+  @Explain(displayName = "name", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  public String getViewName() {
+    return viewName;
+  }
+
+  @Explain(displayName = "original text", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  public String getOriginalText() {
+    return originalText;
+  }
+
+  @Explain(displayName = "expanded text")
+  public String getExpandedText() {
+    return expandedText;
+  }
+
+  @Explain(displayName = "columns")
+  public List<String> getSchemaString() {
+    return Utilities.getFieldSchemaString(schema);
+  }
+
+  public List<FieldSchema> getSchema() {
+    return schema;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AlterViewAsAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AlterViewAsAnalyzer.java
new file mode 100644
index 0000000..f0ece29
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AlterViewAsAnalyzer.java
@@ -0,0 +1,92 @@
+/*
+ * 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.create;
+
+import java.util.ArrayList;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory.DDLType;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.ParseUtils;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+/**
+ * Analyzer for alter view ... as commands.
+ */
+@DDLType(types = HiveParser.TOK_ALTERVIEW_AS)
+public class AlterViewAsAnalyzer extends AbstractCreateViewAnalyzer {
+  public AlterViewAsAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  public void analyzeInternal(ASTNode root) throws SemanticException {
+    TableName viewName = getQualifiedTableName((ASTNode) root.getChild(0));
+    String fqViewName = viewName.getNotEmptyDbTable();
+    LOG.info("Altering the query of view " + fqViewName + " position=" + root.getCharPositionInLine());
+
+    ASTNode select = (ASTNode) root.getChild(1).getChild(0);
+
+    String originalText = ctx.getTokenRewriteStream().toString(select.getTokenStartIndex(), select.getTokenStopIndex());
+
+    SemanticAnalyzer analyzer = analyzeQuery(select, fqViewName);
+
+    schema = new ArrayList<FieldSchema>(analyzer.getResultSchema());
+    ParseUtils.validateColumnNameUniqueness(
+        analyzer.getOriginalResultSchema() == null ? schema : analyzer.getOriginalResultSchema());
+
+    String expandedText = ctx.getTokenRewriteStream().toString(select.getTokenStartIndex(), select.getTokenStopIndex());
+
+    AlterViewAsDesc desc = new AlterViewAsDesc(fqViewName, schema, originalText, expandedText);
+    validateCreateView(desc, analyzer);
+
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
+    DDLUtils.addDbAndTableToOutputs(getDatabase(viewName.getDb()), viewName, TableType.VIRTUAL_VIEW, false,
+        null, outputs);
+  }
+
+  private void validateCreateView(AlterViewAsDesc desc, SemanticAnalyzer analyzer) throws SemanticException {
+    validateTablesUsed(analyzer);
+
+    Table oldView = null;
+    try {
+      oldView = getTable(desc.getViewName(), false);
+    } catch (HiveException e) {
+      throw new SemanticException(e.getMessage(), e);
+    }
+
+    if (oldView == null) {
+      String viewNotExistErrorMsg = "The following view does not exist: " + desc.getViewName();
+      throw new SemanticException( ErrorMsg.ALTER_VIEW_AS_SELECT_NOT_EXIST.getMsg(viewNotExistErrorMsg));
+    }
+
+    validateReplaceWithPartitions(desc.getViewName(), oldView, null);
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AlterViewAsDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AlterViewAsDesc.java
new file mode 100644
index 0000000..0a8e757
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AlterViewAsDesc.java
@@ -0,0 +1,37 @@
+/*
+ * 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.create;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.plan.Explain;
+import org.apache.hadoop.hive.ql.plan.Explain.Level;
+
+/**
+ * DDL task description for ALTER VIEW ... AS ... commands.
+ */
+@Explain(displayName = "Alter View As", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+public class AlterViewAsDesc extends AbstractCreateViewDesc {
+  private static final long serialVersionUID = 1L;
+
+  public AlterViewAsDesc(String viewName, List<FieldSchema> schema, String originalText, String expandedText) {
+    super(viewName, schema, originalText, expandedText);
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AlterViewAsOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AlterViewAsOperation.java
new file mode 100644
index 0000000..d7e26a2
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AlterViewAsOperation.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.create;
+
+import org.apache.hadoop.hive.ql.ddl.DDLOperation;
+import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Table;
+
+/**
+ * Operation process of altering a view with a new query.
+ */
+public class AlterViewAsOperation extends DDLOperation<AlterViewAsDesc> {
+  public AlterViewAsOperation(DDLOperationContext context, AlterViewAsDesc desc) {
+    super(context, desc);
+  }
+
+  @Override
+  public int execute() throws HiveException {
+    Table oldview = context.getDb().getTable(desc.getViewName(), false);
+
+    // replace existing view
+    oldview.setViewOriginalText(desc.getOriginalText());
+    oldview.setViewExpandedText(desc.getExpandedText());
+    oldview.setFields(desc.getSchema());
+    oldview.checkValidity(null);
+
+    context.getDb().alterTable(desc.getViewName(), oldview, false, null, true);
+    DDLUtils.addIfAbsentByName(new WriteEntity(oldview, WriteEntity.WriteType.DDL_NO_LOCK),
+        context.getWork().getOutputs());
+
+    return 0;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateMaterializedViewDesc.java
similarity index 66%
copy from ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewDesc.java
copy to ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateMaterializedViewDesc.java
index d1f3694..52c7bb9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateMaterializedViewDesc.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler;
 import org.apache.hadoop.hive.ql.metadata.Table;
-import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
 import org.apache.hadoop.hive.ql.plan.Explain;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
 import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
@@ -47,10 +46,10 @@ import org.slf4j.LoggerFactory;
 /**
  * DDL task description for CREATE VIEW commands.
  */
-@Explain(displayName = "Create View", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
-public class CreateViewDesc implements DDLDesc, Serializable {
+@Explain(displayName = "Create Materialized View", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+public class CreateMaterializedViewDesc implements DDLDesc, Serializable {
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = LoggerFactory.getLogger(CreateViewDesc.class);
+  private static final Logger LOG = LoggerFactory.getLogger(CreateMaterializedViewDesc.class);
 
   private String viewName;
   private List<FieldSchema> schema;
@@ -58,26 +57,22 @@ public class CreateViewDesc implements DDLDesc, Serializable {
   private Map<String, String> tblProps;
   private List<String> partColNames;
   private boolean ifNotExists;
-  private boolean orReplace;
 
   private String originalText;
   private String expandedText;
   private boolean rewriteEnabled;
   private List<FieldSchema> partCols;
-  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
-  private Set<String> tablesUsed;  // only used for materialized views
-  private List<String> sortColNames;  // only used for materialized views
-  private List<FieldSchema> sortCols;  // only used for materialized views
-  private List<String> distributeColNames;  // only used for materialized views
-  private List<FieldSchema> distributeCols;  // only used for materialized views
-  private ReplicationSpec replicationSpec = null;
+  private String location;
+  private String serde;
+  private String storageHandler;
+  private Map<String, String> serdeProps;
+  private Set<String> tablesUsed;
+  private List<String> sortColNames;
+  private List<FieldSchema> sortCols;
+  private List<String> distributeColNames;
+  private List<FieldSchema> distributeCols;
   private Long initialMmWriteId; // Initial MM write ID for CMV and import.
   // The FSOP configuration for the FSOP that is going to write initial data during cmv.
   // This is not needed beyond compilation, so it is transient.
@@ -87,9 +82,9 @@ public class CreateViewDesc implements DDLDesc, Serializable {
   /**
    * Used to create a materialized view descriptor.
    */
-  public CreateViewDesc(String viewName, List<FieldSchema> schema, String comment, Map<String, String> tblProps,
-      List<String> partColNames, List<String> sortColNames, List<String> distributeColNames,
-      boolean ifNotExists, boolean orReplace, boolean rewriteEnabled, boolean isAlterViewAs,
+  public CreateMaterializedViewDesc(String viewName, List<FieldSchema> schema, String comment,
+      Map<String, String> tblProps, List<String> partColNames, List<String> sortColNames,
+      List<String> distributeColNames, boolean ifNotExists, boolean rewriteEnabled,
       String inputFormat, String outputFormat, String location,
       String serde, String storageHandler, Map<String, String> serdeProps) {
     this.viewName = viewName;
@@ -100,11 +95,8 @@ public class CreateViewDesc implements DDLDesc, Serializable {
     this.sortColNames = sortColNames;
     this.distributeColNames = distributeColNames;
     this.ifNotExists = ifNotExists;
-    this.orReplace = orReplace;
 
-    this.isMaterialized = true;
     this.rewriteEnabled = rewriteEnabled;
-    this.isAlterViewAs = isAlterViewAs;
     this.inputFormat = inputFormat;
     this.outputFormat = outputFormat;
     this.location = location;
@@ -113,28 +105,6 @@ public class CreateViewDesc implements DDLDesc, Serializable {
     this.serdeProps = serdeProps;
   }
 
-  /**
-   * Used to create a view descriptor.
-   */
-  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 serde) {
-    this.viewName = viewName;
-    this.schema = schema;
-    this.comment = comment;
-    this.tblProps = tblProps;
-    this.partColNames = partColNames;
-    this.ifNotExists = ifNotExists;
-    this.orReplace = orReplace;
-
-    this.isAlterViewAs = isAlterViewAs;
-    this.isMaterialized = false;
-    this.rewriteEnabled = false;
-    this.inputFormat = inputFormat;
-    this.outputFormat = outputFormat;
-    this.serde = serde;
-  }
-
   @Explain(displayName = "name", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
   public String getViewName() {
     return viewName;
@@ -287,24 +257,6 @@ public class CreateViewDesc implements DDLDesc, Serializable {
     this.tablesUsed = tablesUsed;
   }
 
-  @Explain(displayName = "replace", displayOnlyOnTrue = true)
-  public boolean isReplace() {
-    return orReplace;
-  }
-
-  public void setReplace(boolean replace) {
-    this.orReplace = replace;
-  }
-
-  @Explain(displayName = "is alter view as select", displayOnlyOnTrue = true)
-  public boolean getIsAlterViewAs() {
-    return isAlterViewAs;
-  }
-
-  public void setIsAlterViewAs(boolean isAlterViewAs) {
-    this.isAlterViewAs = isAlterViewAs;
-  }
-
   public String getInputFormat() {
     return inputFormat;
   }
@@ -321,10 +273,6 @@ public class CreateViewDesc implements DDLDesc, Serializable {
     this.outputFormat = outputFormat;
   }
 
-  public boolean isMaterialized() {
-    return isMaterialized;
-  }
-
   public void setLocation(String location) {
     this.location = location;
   }
@@ -344,24 +292,6 @@ public class CreateViewDesc implements DDLDesc, Serializable {
     return serdeProps;
   }
 
-  /**
-   * @param replicationSpec Sets the replication spec governing this create.
-   * This parameter will have meaningful values only for creates happening as a result of a replication.
-   */
-  public void setReplicationSpec(ReplicationSpec replicationSpec) {
-    this.replicationSpec = replicationSpec;
-  }
-
-  /**
-   * @return what kind of replication spec this create is running under.
-   */
-  public ReplicationSpec getReplicationSpec() {
-    if (replicationSpec == null) {
-      this.replicationSpec = new ReplicationSpec();
-    }
-    return this.replicationSpec;
-  }
-
   public Table toTable(HiveConf conf) throws HiveException {
     String[] names = Utilities.getDbTableName(getViewName());
     String databaseName = names[0];
@@ -370,12 +300,8 @@ public class CreateViewDesc implements DDLDesc, Serializable {
     Table tbl = new Table(databaseName, tableName);
     tbl.setViewOriginalText(getViewOriginalText());
     tbl.setViewExpandedText(getViewExpandedText());
-    if (isMaterialized()) {
-      tbl.setRewriteEnabled(isRewriteEnabled());
-      tbl.setTableType(TableType.MATERIALIZED_VIEW);
-    } else {
-      tbl.setTableType(TableType.VIRTUAL_VIEW);
-    }
+    tbl.setRewriteEnabled(isRewriteEnabled());
+    tbl.setTableType(TableType.MATERIALIZED_VIEW);
     tbl.setSerializationLib(null);
     tbl.clearSerDeInfo();
     tbl.setFields(getSchema());
@@ -408,47 +334,45 @@ public class CreateViewDesc implements DDLDesc, Serializable {
       tbl.setOutputFormatClass(getOutputFormat());
     }
 
-    if (isMaterialized()) {
-      if (getLocation() != null) {
-        tbl.setDataLocation(new Path(getLocation()));
-      }
+    if (getLocation() != null) {
+      tbl.setDataLocation(new Path(getLocation()));
+    }
 
-      if (getStorageHandler() != null) {
-        tbl.setProperty(
-                org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE,
-                getStorageHandler());
-      }
-      HiveStorageHandler storageHandler = tbl.getStorageHandler();
-
-      /*
-       * If the user didn't specify a SerDe, we use the default.
-       */
-      String serDeClassName;
-      if (getSerde() == null) {
-        if (storageHandler == null) {
-          serDeClassName = PlanUtils.getDefaultSerDe().getName();
-          LOG.info("Default to {} for materialized view {}", serDeClassName, getViewName());
-        } else {
-          serDeClassName = storageHandler.getSerDeClass().getName();
-          LOG.info("Use StorageHandler-supplied {} for materialized view {}", serDeClassName, getViewName());
-        }
+    if (getStorageHandler() != null) {
+      tbl.setProperty(
+          org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE,
+          getStorageHandler());
+    }
+    HiveStorageHandler storageHandler = tbl.getStorageHandler();
+
+    /*
+     * If the user didn't specify a SerDe, we use the default.
+     */
+    String serDeClassName;
+    if (getSerde() == null) {
+      if (storageHandler == null) {
+        serDeClassName = PlanUtils.getDefaultSerDe().getName();
+        LOG.info("Default to {} for materialized view {}", serDeClassName, getViewName());
       } else {
-        // let's validate that the serde exists
-        serDeClassName = getSerde();
-        DDLUtils.validateSerDe(serDeClassName, conf);
-      }
-      tbl.setSerializationLib(serDeClassName);
-
-      // To remain consistent, we need to set input and output formats both
-      // at the table level and the storage handler level.
-      tbl.setInputFormatClass(getInputFormat());
-      tbl.setOutputFormatClass(getOutputFormat());
-      if (getInputFormat() != null && !getInputFormat().isEmpty()) {
-        tbl.getSd().setInputFormat(tbl.getInputFormatClass().getName());
-      }
-      if (getOutputFormat() != null && !getOutputFormat().isEmpty()) {
-        tbl.getSd().setOutputFormat(tbl.getOutputFormatClass().getName());
+        serDeClassName = storageHandler.getSerDeClass().getName();
+        LOG.info("Use StorageHandler-supplied {} for materialized view {}", serDeClassName, getViewName());
       }
+    } else {
+      // let's validate that the serde exists
+      serDeClassName = getSerde();
+      DDLUtils.validateSerDe(serDeClassName, conf);
+    }
+    tbl.setSerializationLib(serDeClassName);
+
+    // To remain consistent, we need to set input and output formats both
+    // at the table level and the storage handler level.
+    tbl.setInputFormatClass(getInputFormat());
+    tbl.setOutputFormatClass(getOutputFormat());
+    if (getInputFormat() != null && !getInputFormat().isEmpty()) {
+      tbl.getSd().setInputFormat(tbl.getInputFormatClass().getName());
+    }
+    if (getOutputFormat() != null && !getOutputFormat().isEmpty()) {
+      tbl.getSd().setOutputFormat(tbl.getOutputFormatClass().getName());
     }
 
     if (ownerName != null) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateMaterializedViewOperation.java
similarity index 55%
copy from ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewOperation.java
copy to ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateMaterializedViewOperation.java
index f7952a5..30ff28c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateMaterializedViewOperation.java
@@ -36,8 +36,8 @@ import com.google.common.collect.ImmutableSet;
 /**
  * Operation process of creating a view.
  */
-public class CreateViewOperation extends DDLOperation<CreateViewDesc> {
-  public CreateViewOperation(DDLOperationContext context, CreateViewDesc desc) {
+public class CreateMaterializedViewOperation extends DDLOperation<CreateMaterializedViewDesc> {
+  public CreateMaterializedViewOperation(DDLOperationContext context, CreateMaterializedViewDesc desc) {
     super(context, desc);
   }
 
@@ -45,55 +45,13 @@ public class CreateViewOperation extends DDLOperation<CreateViewDesc> {
   public int execute() throws HiveException {
     Table oldview = context.getDb().getTable(desc.getViewName(), false);
     if (oldview != null) {
-      // Check whether we are replicating
-      if (desc.getReplicationSpec().isInReplicationScope()) {
-        // if this is a replication spec, then replace-mode semantics might apply.
-        if (desc.getReplicationSpec().allowEventReplacementInto(oldview.getParameters())) {
-          desc.setReplace(true); // we replace existing view.
-        } else {
-          LOG.debug("DDLTask: Create View is skipped as view {} is newer than update",
-              desc.getViewName()); // no replacement, the existing table state is newer than our update.
-          return 0;
-        }
-      }
-
-      if (!desc.isReplace()) {
-        if (desc.getIfNotExists()) {
-          return 0;
-        }
 
-        // View already exists, thus we should be replacing
-        throw new HiveException(ErrorMsg.TABLE_ALREADY_EXISTS.getMsg(desc.getViewName()));
+      if (desc.getIfNotExists()) {
+        return 0;
       }
 
-      // It should not be a materialized view
-      assert !desc.isMaterialized();
-
-      // replace existing view
-      // remove the existing partition columns from the field schema
-      oldview.setViewOriginalText(desc.getViewOriginalText());
-      oldview.setViewExpandedText(desc.getViewExpandedText());
-      oldview.setFields(desc.getSchema());
-      if (desc.getComment() != null) {
-        oldview.setProperty("comment", desc.getComment());
-      }
-      if (desc.getTblProps() != null) {
-        oldview.getTTable().getParameters().putAll(desc.getTblProps());
-      }
-      oldview.setPartCols(desc.getPartCols());
-      if (desc.getInputFormat() != null) {
-        oldview.setInputFormatClass(desc.getInputFormat());
-      }
-      if (desc.getOutputFormat() != null) {
-        oldview.setOutputFormatClass(desc.getOutputFormat());
-      }
-      oldview.checkValidity(null);
-      if (desc.getOwnerName() != null) {
-        oldview.setOwner(desc.getOwnerName());
-      }
-      context.getDb().alterTable(desc.getViewName(), oldview, false, null, true);
-      DDLUtils.addIfAbsentByName(new WriteEntity(oldview, WriteEntity.WriteType.DDL_NO_LOCK),
-          context.getWork().getOutputs());
+      // Materialized View already exists, thus we should be replacing
+      throw new HiveException(ErrorMsg.TABLE_ALREADY_EXISTS.getMsg(desc.getViewName()));
     } else {
       // We create new view
       Table tbl = desc.toTable(context.getConf());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewAnalyzer.java
new file mode 100644
index 0000000..69bde92
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewAnalyzer.java
@@ -0,0 +1,214 @@
+/*
+ * 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.create;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory.DDLType;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.HiveUtils;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.ParseUtils;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+/**
+ * Analyzer for create view commands.
+ */
+@DDLType(types = HiveParser.TOK_CREATEVIEW)
+public class CreateViewAnalyzer extends AbstractCreateViewAnalyzer {
+  public CreateViewAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  public void analyzeInternal(ASTNode root) throws SemanticException {
+    TableName viewName = getQualifiedTableName((ASTNode) root.getChild(0));
+    String fqViewName = viewName.getNotEmptyDbTable();
+    LOG.info("Creating view " + fqViewName + " position=" + root.getCharPositionInLine());
+
+    Map<Integer, ASTNode> children = new HashMap<>();
+    for (int i = 1; i < root.getChildCount(); i++) {
+      ASTNode child = (ASTNode) root.getChild(i);
+      children.put(child.getToken().getType(), child);
+    }
+
+    List<FieldSchema> imposedSchema = children.containsKey(HiveParser.TOK_TABCOLNAME) ?
+        getColumns((ASTNode) children.remove(HiveParser.TOK_TABCOLNAME)) : null;
+    boolean ifNotExists = children.remove(HiveParser.TOK_IFNOTEXISTS) != null;
+    boolean orReplace = children.remove(HiveParser.TOK_ORREPLACE) != null;
+    String comment = children.containsKey(HiveParser.TOK_TABLECOMMENT) ?
+        unescapeSQLString(children.remove(HiveParser.TOK_TABLECOMMENT).getChild(0).getText()) : null;
+    ASTNode select = children.remove(HiveParser.TOK_QUERY);
+    Map<String, String> properties = children.containsKey(HiveParser.TOK_TABLEPROPERTIES) ?
+        getProps((ASTNode) children.remove(HiveParser.TOK_TABLEPROPERTIES).getChild(0)) : null;
+    List<String> partitionColumnNames = children.containsKey(HiveParser.TOK_VIEWPARTCOLS) ?
+        getColumnNames((ASTNode) children.remove(HiveParser.TOK_VIEWPARTCOLS).getChild(0)) : null;
+
+    assert children.isEmpty();
+
+    if (ifNotExists && orReplace) {
+      throw new SemanticException("Can't combine IF NOT EXISTS and OR REPLACE.");
+    }
+
+    String originalText = ctx.getTokenRewriteStream().toString(select.getTokenStartIndex(), select.getTokenStopIndex());
+
+    SemanticAnalyzer analyzer = analyzeQuery(select, fqViewName);
+
+    schema = new ArrayList<FieldSchema>(analyzer.getResultSchema());
+    ParseUtils.validateColumnNameUniqueness(
+        analyzer.getOriginalResultSchema() == null ? schema : analyzer.getOriginalResultSchema());
+
+    String expandedText = getExpandedText(imposedSchema, select, viewName);
+
+    List<FieldSchema> partitionColumns = getPartitionColumns(imposedSchema, select, viewName,
+        partitionColumnNames);
+
+    CreateViewDesc desc = new CreateViewDesc(fqViewName, schema, comment, properties, partitionColumnNames,
+        ifNotExists, orReplace, originalText, expandedText, partitionColumns);
+    validateCreateView(desc, analyzer);
+
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
+    DDLUtils.addDbAndTableToOutputs(getDatabase(viewName.getDb()), viewName, TableType.VIRTUAL_VIEW, false,
+        properties, outputs);
+  }
+
+  private String getExpandedText(List<FieldSchema> imposedSchema, ASTNode select, TableName viewName)
+      throws SemanticException {
+    if (imposedSchema != null) {
+      if (imposedSchema.size() != schema.size()) {
+        throw new SemanticException(SemanticAnalyzer.generateErrorMessage(select, ErrorMsg.VIEW_COL_MISMATCH.getMsg()));
+      }
+    }
+
+    String expandedText = ctx.getTokenRewriteStream().toString(select.getTokenStartIndex(), select.getTokenStopIndex());
+
+    if (imposedSchema != null) {
+      // Merge the names from the imposed schema into the types from the derived schema.
+      StringBuilder sb = new StringBuilder();
+      sb.append("SELECT ");
+      for (int i = 0; i < schema.size(); i++) {
+        if (i > 0) {
+          sb.append(", ");
+        }
+        // Modify a copy, not the original
+        FieldSchema fieldSchema = new FieldSchema(schema.get(i));
+        // TODO: there's a potential problem here if some table uses external schema like Avro,
+        //       with a very large type name. It seems like the view does not derive the SerDe from
+        //       the table, so it won't be able to just get the type from the deserializer like the
+        //       table does; we won't be able to properly store the type in the RDBMS metastore.
+        //       Not sure if these large cols could be in resultSchema. Ignore this for now 0_o
+        schema.set(i, fieldSchema);
+        sb.append(HiveUtils.unparseIdentifier(fieldSchema.getName(), conf));
+        sb.append(" AS ");
+        String imposedName = imposedSchema.get(i).getName();
+        sb.append(HiveUtils.unparseIdentifier(imposedName, conf));
+        fieldSchema.setName(imposedName);
+        // We don't currently allow imposition of a type
+        fieldSchema.setComment(imposedSchema.get(i).getComment());
+      }
+      sb.append(" FROM (");
+      sb.append(expandedText);
+      sb.append(") ");
+      sb.append(HiveUtils.unparseIdentifier(viewName.getTable(), conf));
+
+      expandedText = sb.toString();
+    }
+    return expandedText;
+  }
+
+  private List<FieldSchema> getPartitionColumns(List<FieldSchema> imposedSchema, ASTNode select, TableName viewName,
+      List<String> partitionColumnNames) throws SemanticException {
+    if (partitionColumnNames == null) {
+      return null;
+    }
+
+    // Make sure all partitioning columns referenced actually exist and are in the correct order at the end of the
+    // list of columns produced by the view. Also move the field schema descriptors from derivedSchema to the
+    // partitioning key descriptor.
+    if (partitionColumnNames.size() > schema.size()) {
+      throw new SemanticException(ErrorMsg.VIEW_PARTITION_MISMATCH.getMsg());
+    }
+
+    // Get the partition columns from the end of derivedSchema.
+    List<FieldSchema> partitionColumns =
+        schema.subList(schema.size() - partitionColumnNames.size(), schema.size());
+
+    // Verify that the names match the PARTITIONED ON clause.
+    Iterator<String> columnNameIterator = partitionColumnNames.iterator();
+    Iterator<FieldSchema> schemaIterator = partitionColumns.iterator();
+    while (columnNameIterator.hasNext()) {
+      String columnName = columnNameIterator.next();
+      FieldSchema fieldSchema = schemaIterator.next();
+      if (!fieldSchema.getName().equals(columnName)) {
+        throw new SemanticException(ErrorMsg.VIEW_PARTITION_MISMATCH.getMsg());
+      }
+    }
+
+    // Boundary case: require at least one non-partitioned column for consistency with tables.
+    if (partitionColumnNames.size() == schema.size()) {
+      throw new SemanticException(ErrorMsg.VIEW_PARTITION_TOTAL.getMsg());
+    }
+
+    // Now make a copy, and remove the partition columns from the end of derivedSchema.
+    // (Clearing the subList writes through to the underlying derivedSchema ArrayList.)
+    List<FieldSchema> partitionColumnsCopy = new ArrayList<FieldSchema>(partitionColumns);
+    partitionColumns.clear();
+    return partitionColumnsCopy;
+  }
+
+  private void validateCreateView(CreateViewDesc desc, SemanticAnalyzer analyzer) throws SemanticException {
+    try {
+      validateTablesUsed(analyzer);
+
+      //replace view
+      Table oldView = getTable(desc.getViewName(), false);
+      if (desc.isReplace() && oldView != null) {
+        if (oldView.getTableType().equals(TableType.MATERIALIZED_VIEW)) {
+          throw new SemanticException(ErrorMsg.REPLACE_MATERIALIZED_WITH_VIEW, oldView.getTableName());
+        }
+
+        // Existing table is not a view
+        if (!oldView.getTableType().equals(TableType.VIRTUAL_VIEW) &&
+            !oldView.getTableType().equals(TableType.MATERIALIZED_VIEW)) {
+          String tableNotViewErrorMsg = "The following is an existing table, not a view: " + desc.getViewName();
+          throw new SemanticException(ErrorMsg.EXISTING_TABLE_IS_NOT_VIEW.getMsg(tableNotViewErrorMsg));
+        }
+
+        validateReplaceWithPartitions(desc.getViewName(), oldView, desc.getPartitionColumns());
+      }
+    } catch (HiveException e) {
+      throw new SemanticException(e.getMessage(), e);
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewDesc.java
index d1f3694..e71cbce 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewDesc.java
@@ -18,238 +18,55 @@
 
 package org.apache.hadoop.hive.ql.ddl.view.create;
 
-import java.io.Serializable;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
-import org.apache.commons.collections.CollectionUtils;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.StatsSetupConst;
-import org.apache.hadoop.hive.conf.Constants;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.ql.ddl.DDLDesc;
-import org.apache.hadoop.hive.ql.ddl.DDLUtils;
 import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler;
-import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
 import org.apache.hadoop.hive.ql.plan.Explain;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
-import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
-import org.apache.hadoop.hive.ql.plan.PlanUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * DDL task description for CREATE VIEW commands.
  */
 @Explain(displayName = "Create View", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
-public class CreateViewDesc implements DDLDesc, Serializable {
+public class CreateViewDesc extends AbstractCreateViewDesc {
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = LoggerFactory.getLogger(CreateViewDesc.class);
 
-  private String viewName;
-  private List<FieldSchema> schema;
-  private String comment;
-  private Map<String, String> tblProps;
-  private List<String> partColNames;
-  private boolean ifNotExists;
-  private boolean orReplace;
+  private final String comment;
+  private final Map<String, String> properties;
+  private final List<String> partitionColumnNames;
+  private final boolean ifNotExists;
+  private final boolean replace;
+  private final List<FieldSchema> partitionColumns;
 
-  private String originalText;
-  private String expandedText;
-  private boolean rewriteEnabled;
-  private List<FieldSchema> partCols;
-  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
-  private Set<String> tablesUsed;  // only used for materialized views
-  private List<String> sortColNames;  // only used for materialized views
-  private List<FieldSchema> sortCols;  // only used for materialized views
-  private List<String> distributeColNames;  // only used for materialized views
-  private List<FieldSchema> distributeCols;  // only used for materialized views
   private ReplicationSpec replicationSpec = null;
-  private Long initialMmWriteId; // Initial MM write ID for CMV and import.
-  // The FSOP configuration for the FSOP that is going to write initial data during cmv.
-  // This is not needed beyond compilation, so it is transient.
-  private transient FileSinkDesc writer;
   private String ownerName = null;
 
-  /**
-   * Used to create a materialized view descriptor.
-   */
-  public CreateViewDesc(String viewName, List<FieldSchema> schema, String comment, Map<String, String> tblProps,
-      List<String> partColNames, List<String> sortColNames, List<String> distributeColNames,
-      boolean ifNotExists, boolean orReplace, boolean rewriteEnabled, boolean isAlterViewAs,
-      String inputFormat, String outputFormat, String location,
-      String serde, String storageHandler, Map<String, String> serdeProps) {
-    this.viewName = viewName;
-    this.schema = schema;
+  public CreateViewDesc(String viewName, List<FieldSchema> schema, String comment, Map<String, String> properties,
+      List<String> partitionColumnNames, boolean ifNotExists, boolean replace, String originalText,
+      String expandedText, List<FieldSchema> partitionColumns) {
+    super(viewName, schema, originalText, expandedText);
     this.comment = comment;
-    this.tblProps = tblProps;
-    this.partColNames = partColNames;
-    this.sortColNames = sortColNames;
-    this.distributeColNames = distributeColNames;
+    this.properties = properties;
+    this.partitionColumnNames = partitionColumnNames;
     this.ifNotExists = ifNotExists;
-    this.orReplace = orReplace;
-
-    this.isMaterialized = true;
-    this.rewriteEnabled = rewriteEnabled;
-    this.isAlterViewAs = isAlterViewAs;
-    this.inputFormat = inputFormat;
-    this.outputFormat = outputFormat;
-    this.location = location;
-    this.serde = serde;
-    this.storageHandler = storageHandler;
-    this.serdeProps = serdeProps;
-  }
-
-  /**
-   * Used to create a view descriptor.
-   */
-  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 serde) {
-    this.viewName = viewName;
-    this.schema = schema;
-    this.comment = comment;
-    this.tblProps = tblProps;
-    this.partColNames = partColNames;
-    this.ifNotExists = ifNotExists;
-    this.orReplace = orReplace;
-
-    this.isAlterViewAs = isAlterViewAs;
-    this.isMaterialized = false;
-    this.rewriteEnabled = false;
-    this.inputFormat = inputFormat;
-    this.outputFormat = outputFormat;
-    this.serde = serde;
-  }
-
-  @Explain(displayName = "name", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
-  public String getViewName() {
-    return viewName;
-  }
-
-  public void setViewName(String viewName) {
-    this.viewName = viewName;
-  }
-
-  @Explain(displayName = "original text", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
-  public String getViewOriginalText() {
-    return originalText;
-  }
-
-  public void setViewOriginalText(String originalText) {
-    this.originalText = originalText;
-  }
-
-  @Explain(displayName = "expanded text")
-  public String getViewExpandedText() {
-    return expandedText;
-  }
-
-  public void setViewExpandedText(String expandedText) {
-    this.expandedText = expandedText;
-  }
-
-  @Explain(displayName = "rewrite enabled", displayOnlyOnTrue = true)
-  public boolean isRewriteEnabled() {
-    return rewriteEnabled;
-  }
-
-  public void setRewriteEnabled(boolean rewriteEnabled) {
-    this.rewriteEnabled = rewriteEnabled;
-  }
-
-  @Explain(displayName = "columns")
-  public List<String> getSchemaString() {
-    return Utilities.getFieldSchemaString(schema);
-  }
-
-  public List<FieldSchema> getSchema() {
-    return schema;
-  }
-
-  public void setSchema(List<FieldSchema> schema) {
-    this.schema = schema;
+    this.replace = replace;
+    this.partitionColumns = partitionColumns;
   }
 
   @Explain(displayName = "partition columns")
   public List<String> getPartColsString() {
-    return Utilities.getFieldSchemaString(partCols);
+    return Utilities.getFieldSchemaString(partitionColumns);
   }
 
-  public List<FieldSchema> getPartCols() {
-    return partCols;
-  }
-
-  public void setPartCols(List<FieldSchema> partCols) {
-    this.partCols = partCols;
+  public List<FieldSchema> getPartitionColumns() {
+    return partitionColumns;
   }
 
   public List<String> getPartColNames() {
-    return partColNames;
-  }
-
-  public void setPartColNames(List<String> partColNames) {
-    this.partColNames = partColNames;
-  }
-
-  public boolean isOrganized() {
-    return (sortColNames != null && !sortColNames.isEmpty()) ||
-        (distributeColNames != null && !distributeColNames.isEmpty());
-  }
-
-  @Explain(displayName = "sort columns")
-  public List<String> getSortColsString() {
-    return Utilities.getFieldSchemaString(sortCols);
-  }
-
-  public List<FieldSchema> getSortCols() {
-    return sortCols;
-  }
-
-  public void setSortCols(List<FieldSchema> sortCols) {
-    this.sortCols = sortCols;
-  }
-
-  public List<String> getSortColNames() {
-    return sortColNames;
-  }
-
-  public void setSortColNames(List<String> sortColNames) {
-    this.sortColNames = sortColNames;
-  }
-
-  @Explain(displayName = "distribute columns")
-  public List<String> getDistributeColsString() {
-    return Utilities.getFieldSchemaString(distributeCols);
-  }
-
-  public List<FieldSchema> getDistributeCols() {
-    return distributeCols;
-  }
-
-  public void setDistributeCols(List<FieldSchema> distributeCols) {
-    this.distributeCols = distributeCols;
-  }
-
-  public List<String> getDistributeColNames() {
-    return distributeColNames;
-  }
-
-  public void setDistributeColNames(List<String> distributeColNames) {
-    this.distributeColNames = distributeColNames;
+    return partitionColumnNames;
   }
 
   @Explain(displayName = "comment")
@@ -257,17 +74,9 @@ public class CreateViewDesc implements DDLDesc, Serializable {
     return comment;
   }
 
-  public void setComment(String comment) {
-    this.comment = comment;
-  }
-
-  public void setTblProps(Map<String, String> tblProps) {
-    this.tblProps = tblProps;
-  }
-
-  @Explain(displayName = "table properties")
-  public Map<String, String> getTblProps() {
-    return tblProps;
+  @Explain(displayName = "properties")
+  public Map<String, String> getProperties() {
+    return properties;
   }
 
   @Explain(displayName = "if not exists", displayOnlyOnTrue = true)
@@ -275,73 +84,9 @@ public class CreateViewDesc implements DDLDesc, Serializable {
     return ifNotExists;
   }
 
-  public void setIfNotExists(boolean ifNotExists) {
-    this.ifNotExists = ifNotExists;
-  }
-
-  public Set<String> getTablesUsed() {
-    return tablesUsed;
-  }
-
-  public void setTablesUsed(Set<String> tablesUsed) {
-    this.tablesUsed = tablesUsed;
-  }
-
   @Explain(displayName = "replace", displayOnlyOnTrue = true)
   public boolean isReplace() {
-    return orReplace;
-  }
-
-  public void setReplace(boolean replace) {
-    this.orReplace = replace;
-  }
-
-  @Explain(displayName = "is alter view as select", displayOnlyOnTrue = true)
-  public boolean getIsAlterViewAs() {
-    return isAlterViewAs;
-  }
-
-  public void setIsAlterViewAs(boolean isAlterViewAs) {
-    this.isAlterViewAs = isAlterViewAs;
-  }
-
-  public String getInputFormat() {
-    return inputFormat;
-  }
-
-  public void setInputFormat(String inputFormat) {
-    this.inputFormat = inputFormat;
-  }
-
-  public String getOutputFormat() {
-    return outputFormat;
-  }
-
-  public void setOutputFormat(String outputFormat) {
-    this.outputFormat = outputFormat;
-  }
-
-  public boolean isMaterialized() {
-    return isMaterialized;
-  }
-
-  public void setLocation(String location) {
-    this.location = location;
-  }
-  public String getLocation() {
-    return location;
-  }
-
-  public String getSerde() {
-    return serde;
-  }
-
-  public String getStorageHandler() {
-    return storageHandler;
-  }
-
-  public Map<String, String> getSerdeProps() {
-    return serdeProps;
+    return replace;
   }
 
   /**
@@ -359,126 +104,7 @@ public class CreateViewDesc implements DDLDesc, Serializable {
     if (replicationSpec == null) {
       this.replicationSpec = new ReplicationSpec();
     }
-    return this.replicationSpec;
-  }
-
-  public Table toTable(HiveConf conf) throws HiveException {
-    String[] names = Utilities.getDbTableName(getViewName());
-    String databaseName = names[0];
-    String tableName = names[1];
-
-    Table tbl = new Table(databaseName, tableName);
-    tbl.setViewOriginalText(getViewOriginalText());
-    tbl.setViewExpandedText(getViewExpandedText());
-    if (isMaterialized()) {
-      tbl.setRewriteEnabled(isRewriteEnabled());
-      tbl.setTableType(TableType.MATERIALIZED_VIEW);
-    } else {
-      tbl.setTableType(TableType.VIRTUAL_VIEW);
-    }
-    tbl.setSerializationLib(null);
-    tbl.clearSerDeInfo();
-    tbl.setFields(getSchema());
-    if (getComment() != null) {
-      tbl.setProperty("comment", getComment());
-    }
-
-    if (tblProps != null) {
-      tbl.getParameters().putAll(tblProps);
-    }
-
-    if (!CollectionUtils.isEmpty(partCols)) {
-      tbl.setPartCols(partCols);
-    }
-
-    if (!CollectionUtils.isEmpty(sortColNames)) {
-      tbl.setProperty(Constants.MATERIALIZED_VIEW_SORT_COLUMNS,
-          Utilities.encodeColumnNames(sortColNames));
-    }
-    if (!CollectionUtils.isEmpty(distributeColNames)) {
-      tbl.setProperty(Constants.MATERIALIZED_VIEW_DISTRIBUTE_COLUMNS,
-          Utilities.encodeColumnNames(distributeColNames));
-    }
-
-    if (getInputFormat() != null) {
-      tbl.setInputFormatClass(getInputFormat());
-    }
-
-    if (getOutputFormat() != null) {
-      tbl.setOutputFormatClass(getOutputFormat());
-    }
-
-    if (isMaterialized()) {
-      if (getLocation() != null) {
-        tbl.setDataLocation(new Path(getLocation()));
-      }
-
-      if (getStorageHandler() != null) {
-        tbl.setProperty(
-                org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE,
-                getStorageHandler());
-      }
-      HiveStorageHandler storageHandler = tbl.getStorageHandler();
-
-      /*
-       * If the user didn't specify a SerDe, we use the default.
-       */
-      String serDeClassName;
-      if (getSerde() == null) {
-        if (storageHandler == null) {
-          serDeClassName = PlanUtils.getDefaultSerDe().getName();
-          LOG.info("Default to {} for materialized view {}", serDeClassName, getViewName());
-        } else {
-          serDeClassName = storageHandler.getSerDeClass().getName();
-          LOG.info("Use StorageHandler-supplied {} for materialized view {}", serDeClassName, getViewName());
-        }
-      } else {
-        // let's validate that the serde exists
-        serDeClassName = getSerde();
-        DDLUtils.validateSerDe(serDeClassName, conf);
-      }
-      tbl.setSerializationLib(serDeClassName);
-
-      // To remain consistent, we need to set input and output formats both
-      // at the table level and the storage handler level.
-      tbl.setInputFormatClass(getInputFormat());
-      tbl.setOutputFormatClass(getOutputFormat());
-      if (getInputFormat() != null && !getInputFormat().isEmpty()) {
-        tbl.getSd().setInputFormat(tbl.getInputFormatClass().getName());
-      }
-      if (getOutputFormat() != null && !getOutputFormat().isEmpty()) {
-        tbl.getSd().setOutputFormat(tbl.getOutputFormatClass().getName());
-      }
-    }
-
-    if (ownerName != null) {
-      tbl.setOwner(ownerName);
-    }
-
-    // Sets the column state for the create view statement (false since it is a creation).
-    // Similar to logic in CreateTableDesc.
-    StatsSetupConst.setStatsStateForCreateTable(tbl.getTTable().getParameters(), null,
-        StatsSetupConst.FALSE);
-
-    return tbl;
-  }
-
-  public void setInitialMmWriteId(Long mmWriteId) {
-    this.initialMmWriteId = mmWriteId;
-  }
-
-  public Long getInitialMmWriteId() {
-    return initialMmWriteId;
-  }
-
-  public FileSinkDesc getAndUnsetWriter() {
-    FileSinkDesc fsd = writer;
-    writer = null;
-    return fsd;
-  }
-
-  public void setWriter(FileSinkDesc writer) {
-    this.writer = writer;
+    return replicationSpec;
   }
 
   public void setOwnerName(String ownerName) {
@@ -486,7 +112,6 @@ public class CreateViewDesc implements DDLDesc, Serializable {
   }
 
   public String getOwnerName() {
-    return this.ownerName;
+    return ownerName;
   }
-
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewOperation.java
index f7952a5..4be0f5a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/CreateViewOperation.java
@@ -18,10 +18,11 @@
 
 package org.apache.hadoop.hive.ql.ddl.view.create;
 
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
-import org.apache.hadoop.hive.metastore.api.CreationMetadata;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.ddl.DDLOperation;
 import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
@@ -30,8 +31,8 @@ import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.hooks.LineageInfo.DataContainer;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
-
-import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.hive.ql.parse.HiveTableName;
+import org.apache.hadoop.hive.ql.parse.StorageFormat;
 
 /**
  * Operation process of creating a view.
@@ -45,19 +46,20 @@ public class CreateViewOperation extends DDLOperation<CreateViewDesc> {
   public int execute() throws HiveException {
     Table oldview = context.getDb().getTable(desc.getViewName(), false);
     if (oldview != null) {
+      boolean isReplace = desc.isReplace();
+
       // Check whether we are replicating
       if (desc.getReplicationSpec().isInReplicationScope()) {
         // if this is a replication spec, then replace-mode semantics might apply.
         if (desc.getReplicationSpec().allowEventReplacementInto(oldview.getParameters())) {
-          desc.setReplace(true); // we replace existing view.
+          isReplace = true; // we replace existing view.
         } else {
-          LOG.debug("DDLTask: Create View is skipped as view {} is newer than update",
-              desc.getViewName()); // no replacement, the existing table state is newer than our update.
+          LOG.debug("DDLTask: Create View is skipped as view {} is newer than update", desc.getViewName());
           return 0;
         }
       }
 
-      if (!desc.isReplace()) {
+      if (!isReplace) {
         if (desc.getIfNotExists()) {
           return 0;
         }
@@ -66,27 +68,19 @@ public class CreateViewOperation extends DDLOperation<CreateViewDesc> {
         throw new HiveException(ErrorMsg.TABLE_ALREADY_EXISTS.getMsg(desc.getViewName()));
       }
 
-      // It should not be a materialized view
-      assert !desc.isMaterialized();
-
       // replace existing view
       // remove the existing partition columns from the field schema
-      oldview.setViewOriginalText(desc.getViewOriginalText());
-      oldview.setViewExpandedText(desc.getViewExpandedText());
+      oldview.setViewOriginalText(desc.getOriginalText());
+      oldview.setViewExpandedText(desc.getExpandedText());
       oldview.setFields(desc.getSchema());
       if (desc.getComment() != null) {
         oldview.setProperty("comment", desc.getComment());
       }
-      if (desc.getTblProps() != null) {
-        oldview.getTTable().getParameters().putAll(desc.getTblProps());
-      }
-      oldview.setPartCols(desc.getPartCols());
-      if (desc.getInputFormat() != null) {
-        oldview.setInputFormatClass(desc.getInputFormat());
-      }
-      if (desc.getOutputFormat() != null) {
-        oldview.setOutputFormatClass(desc.getOutputFormat());
+      if (desc.getProperties() != null) {
+        oldview.getTTable().getParameters().putAll(desc.getProperties());
       }
+      oldview.setPartCols(desc.getPartitionColumns());
+
       oldview.checkValidity(null);
       if (desc.getOwnerName() != null) {
         oldview.setOwner(desc.getOwnerName());
@@ -96,23 +90,54 @@ public class CreateViewOperation extends DDLOperation<CreateViewDesc> {
           context.getWork().getOutputs());
     } else {
       // We create new view
-      Table tbl = desc.toTable(context.getConf());
-      // We set the signature for the view if it is a materialized view
-      if (tbl.isMaterializedView()) {
-        CreationMetadata cm =
-            new CreationMetadata(MetaStoreUtils.getDefaultCatalog(context.getConf()), tbl.getDbName(),
-                tbl.getTableName(), ImmutableSet.copyOf(desc.getTablesUsed()));
-        cm.setValidTxnList(context.getConf().get(ValidTxnWriteIdList.VALID_TABLES_WRITEIDS_KEY));
-        tbl.getTTable().setCreationMetadata(cm);
-      }
-      context.getDb().createTable(tbl, desc.getIfNotExists());
-      DDLUtils.addIfAbsentByName(new WriteEntity(tbl, WriteEntity.WriteType.DDL_NO_LOCK),
+      Table view = createViewObject();
+      context.getDb().createTable(view, desc.getIfNotExists());
+      DDLUtils.addIfAbsentByName(new WriteEntity(view, WriteEntity.WriteType.DDL_NO_LOCK),
           context.getWork().getOutputs());
 
       //set lineage info
-      DataContainer dc = new DataContainer(tbl.getTTable());
-      context.getQueryState().getLineageState().setLineage(new Path(desc.getViewName()), dc, tbl.getCols());
+      DataContainer dc = new DataContainer(view.getTTable());
+      context.getQueryState().getLineageState().setLineage(new Path(desc.getViewName()), dc, view.getCols());
     }
     return 0;
   }
+
+  private Table createViewObject() throws HiveException {
+    TableName name = HiveTableName.of(desc.getViewName());
+    Table view = new Table(name.getDb(), name.getTable());
+    view.setViewOriginalText(desc.getOriginalText());
+    view.setViewExpandedText(desc.getExpandedText());
+    view.setTableType(TableType.VIRTUAL_VIEW);
+    view.setSerializationLib(null);
+    view.clearSerDeInfo();
+    view.setFields(desc.getSchema());
+    if (desc.getComment() != null) {
+      view.setProperty("comment", desc.getComment());
+    }
+
+    if (desc.getProperties() != null) {
+      view.getParameters().putAll(desc.getProperties());
+    }
+
+    if (!CollectionUtils.isEmpty(desc.getPartitionColumns())) {
+      view.setPartCols(desc.getPartitionColumns());
+    }
+
+    StorageFormat storageFormat = new StorageFormat(context.getConf());
+    storageFormat.fillDefaultStorageFormat(false, false);
+
+    view.setInputFormatClass(storageFormat.getInputFormat());
+    view.setOutputFormatClass(storageFormat.getOutputFormat());
+
+    if (desc.getOwnerName() != null) {
+      view.setOwner(desc.getOwnerName());
+    }
+
+    // Sets the column state for the create view statement (false since it is a creation).
+    // Similar to logic in CreateTableDesc.
+    StatsSetupConst.setStatsStateForCreateTable(view.getTTable().getParameters(), null,
+        StatsSetupConst.FALSE);
+
+    return view;
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java
index a48cd5b..7ebc7cf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java
@@ -381,21 +381,17 @@ public class ReplLoadTask extends Task<ReplLoadWork> implements Serializable {
     String dbName = dbNameToLoadIn == null ? table.getDbName() : dbNameToLoadIn;
     TableName tableName = HiveTableName.ofNullable(table.getTableName(), dbName);
     String dbDotView = tableName.getNotEmptyDbTable();
-    CreateViewDesc desc = new CreateViewDesc(dbDotView, table.getAllCols(), null, table.getParameters(),
-        table.getPartColNames(), false, false, false, table.getSd().getInputFormat(),
-        table.getSd().getOutputFormat(),
-        table.getSd().getSerdeInfo().getSerializationLib());
-    String originalText = table.getViewOriginalText();
-    String expandedText = table.getViewExpandedText();
 
+    String viewOriginalText = table.getViewOriginalText();
+    String viewExpandedText = table.getViewExpandedText();
     if (!dbName.equals(table.getDbName())) {
       // TODO: If the DB name doesn't match with the metadata from dump, then need to rewrite the original and expanded
       // texts using new DB name. Currently it refers to the source database name.
     }
 
-    desc.setViewOriginalText(originalText);
-    desc.setViewExpandedText(expandedText);
-    desc.setPartCols(table.getPartCols());
+    CreateViewDesc desc = new CreateViewDesc(dbDotView, table.getAllCols(), null, table.getParameters(),
+        table.getPartColNames(), false, false, viewOriginalText, viewExpandedText, table.getPartCols());
+
     desc.setReplicationSpec(metaData.getReplicationSpec());
     desc.setOwnerName(table.getOwner());
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
index 36d6c97..85d8531 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
@@ -68,7 +68,7 @@ import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.ddl.table.create.CreateTableDesc;
-import org.apache.hadoop.hive.ql.ddl.view.create.CreateViewDesc;
+import org.apache.hadoop.hive.ql.ddl.view.create.CreateMaterializedViewDesc;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.hooks.Entity;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
@@ -1969,7 +1969,7 @@ public class AcidUtils {
     return tableIsTransactional != null && tableIsTransactional.equalsIgnoreCase("true");
   }
 
-  public static boolean isTransactionalView(CreateViewDesc view) {
+  public static boolean isTransactionalView(CreateMaterializedViewDesc view) {
     if (view == null || view.getTblProps() == null) {
       return false;
     }
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 4222a9b..088efd4 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
@@ -567,6 +567,12 @@ public class CalcitePlanner extends SemanticAnalyzer {
             }
 
             // 2. Regen OP plan from optimized AST
+            if (forViewCreation) {
+              // the reset would remove the translations
+              executeUnparseTranlations();
+              // save the resultSchema before rewriting it
+              originalResultSchema = resultSchema;
+            }
             if (cboCtx.type == PreCboCtx.Type.VIEW) {
               try {
                 viewSelect = handleCreateViewDDL(newAST);
@@ -1849,7 +1855,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         calciteGenPlan = genLogicalPlan(getQB(), true, null, null);
         // if it is to create view, we do not use table alias
         resultSchema = convertRowSchemaToResultSetSchema(relToHiveRR.get(calciteGenPlan),
-            getQB().isView() || getQB().isMaterializedView() ? false : HiveConf.getBoolVar(conf,
+            (forViewCreation || getQB().isMaterializedView()) ? false : HiveConf.getBoolVar(conf,
                 HiveConf.ConfVars.HIVE_RESULTSET_USE_UNIQUE_COLUMN_NAMES));
       } catch (SemanticException e) {
         semanticException = e;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
index da443f4..99b0225 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.QueryProperties;
 import org.apache.hadoop.hive.ql.QueryState;
 import org.apache.hadoop.hive.ql.ddl.table.create.CreateTableDesc;
-import org.apache.hadoop.hive.ql.ddl.view.create.CreateViewDesc;
+import org.apache.hadoop.hive.ql.ddl.view.create.CreateMaterializedViewDesc;
 import org.apache.hadoop.hive.ql.ddl.view.materialized.update.MaterializedViewUpdateDesc;
 import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
@@ -118,7 +118,7 @@ public class ParseContext {
 
   private AnalyzeRewriteContext analyzeRewrite;
   private CreateTableDesc createTableDesc;
-  private CreateViewDesc createViewDesc;
+  private CreateMaterializedViewDesc createViewDesc;
   private MaterializedViewUpdateDesc materializedViewUpdateDesc;
   private boolean reduceSinkAddedBySortedDynPartition;
 
@@ -193,7 +193,7 @@ public class ParseContext {
       Map<String, ReadEntity> viewAliasToInput,
       List<ReduceSinkOperator> reduceSinkOperatorsAddedByEnforceBucketingSorting,
       AnalyzeRewriteContext analyzeRewrite, CreateTableDesc createTableDesc,
-      CreateViewDesc createViewDesc, MaterializedViewUpdateDesc materializedViewUpdateDesc,
+      CreateMaterializedViewDesc createViewDesc, MaterializedViewUpdateDesc materializedViewUpdateDesc,
       QueryProperties queryProperties,
       Map<SelectOperator, Table> viewProjectToTableSchema) {
     this.queryState = queryState;
@@ -591,7 +591,7 @@ public class ParseContext {
     this.createTableDesc = createTableDesc;
   }
 
-  public CreateViewDesc getCreateViewDesc() {
+  public CreateMaterializedViewDesc getCreateViewDesc() {
     return createViewDesc;
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java
index 9d94b6e..c5f2ef3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.ql.parse;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
@@ -29,9 +28,8 @@ import java.util.Set;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.ddl.table.create.CreateTableDesc;
-import org.apache.hadoop.hive.ql.ddl.view.create.CreateViewDesc;
+import org.apache.hadoop.hive.ql.ddl.view.create.CreateMaterializedViewDesc;
 import org.apache.hadoop.hive.ql.metadata.Table;
 
 /**
@@ -64,7 +62,7 @@ public class QB {
   private Set<String> aliasInsideView;
 
   // If this is a materialized view, this stores the view descriptor
-  private CreateViewDesc viewDesc;
+  private CreateMaterializedViewDesc viewDesc;
 
   // used by PTFs
   /*
@@ -407,20 +405,16 @@ public class QB {
     return havingClauseSubQueryPredicate;
   }
 
-  public CreateViewDesc getViewDesc() {
+  public CreateMaterializedViewDesc getViewDesc() {
     return viewDesc;
   }
 
-  public void setViewDesc(CreateViewDesc viewDesc) {
+  public void setViewDesc(CreateMaterializedViewDesc viewDesc) {
     this.viewDesc = viewDesc;
   }
 
   public boolean isMaterializedView() {
-    return viewDesc != null && viewDesc.isMaterialized();
-  }
-
-  public boolean isView() {
-    return viewDesc != null && !viewDesc.isMaterialized();
+    return viewDesc != null;
   }
 
   public boolean isMultiDestQuery() {
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 46e1f42..aeaf8ac 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
@@ -111,7 +111,7 @@ import org.apache.hadoop.hive.ql.ddl.table.create.like.CreateTableLikeDesc;
 import org.apache.hadoop.hive.ql.ddl.table.misc.preinsert.PreInsertTableDesc;
 import org.apache.hadoop.hive.ql.ddl.table.misc.properties.AlterTableUnsetPropertiesDesc;
 import org.apache.hadoop.hive.ql.ddl.table.storage.skewed.SkewedTableUtils;
-import org.apache.hadoop.hive.ql.ddl.view.create.CreateViewDesc;
+import org.apache.hadoop.hive.ql.ddl.view.create.CreateMaterializedViewDesc;
 import org.apache.hadoop.hive.ql.ddl.view.materialized.update.MaterializedViewUpdateDesc;
 import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.ArchiveUtils;
@@ -339,6 +339,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   private Map<TableScanOperator, SampleDesc> opToSamplePruner;
   private final Map<TableScanOperator, Map<String, ExprNodeDesc>> opToPartToSkewedPruner;
   private Map<SelectOperator, Table> viewProjectToTableSchema;
+  private Operator<? extends OperatorDesc> sinkOp;
   /**
    * a map for the split sampling, from alias to an instance of SplitSample
    * that describes percentage and number.
@@ -347,7 +348,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   private final Map<GroupByOperator, Set<String>> groupOpToInputTables;
   protected Map<String, PrunedPartitionList> prunedPartitions;
   protected List<FieldSchema> resultSchema;
-  protected CreateViewDesc createVwDesc;
+  protected List<FieldSchema> originalResultSchema;
+  protected CreateMaterializedViewDesc createVwDesc;
   private MaterializedViewUpdateDesc materializedViewUpdateDesc;
   private List<String> viewsExpanded;
   protected ASTNode viewSelect;
@@ -371,6 +373,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   // flag for no scan during analyze ... compute statistics
   private boolean noscan;
 
+  // flag indicating that the analyzations should go only till resultSchema is ready
+  protected boolean forViewCreation;
+  private String fqViewName;
+
   // whether this is a mv rebuild rewritten expression
   protected MaterializationRebuildMode mvRebuildMode = MaterializationRebuildMode.NONE;
   protected String mvRebuildDbName; // Db name for materialization to rebuild
@@ -7526,7 +7532,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       TableName tableName = null;
       Map<String, String> tblProps = null;
       CreateTableDesc tblDesc = qb.getTableDesc();
-      CreateViewDesc viewDesc = qb.getViewDesc();
+      CreateMaterializedViewDesc viewDesc = qb.getViewDesc();
       if (tblDesc != null) {
         fieldSchemas = new ArrayList<>();
         partitionColumns = new ArrayList<>();
@@ -11980,7 +11986,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
   }
 
-  private Table getTableObjectByName(String tableName) throws HiveException {
+  public Table getTableObjectByName(String tableName) throws HiveException {
     return getTableObjectByName(tableName, true);
   }
 
@@ -12265,10 +12271,7 @@ 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_ALTERVIEW &&
-            ast.getChild(1).getType() == HiveParser.TOK_QUERY)) {
+    if (ast.getToken().getType() == HiveParser.TOK_CREATE_MATERIALIZED_VIEW) {
       child = analyzeCreateView(ast, qb, plannerCtx);
       if (child == null) {
         return false;
@@ -12278,6 +12281,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       viewsExpanded.add(createVwDesc.getViewName());
     }
 
+    if (forViewCreation) {
+      viewsExpanded.add(fqViewName);
+    }
+
     switch(ast.getToken().getType()) {
     case HiveParser.TOK_SET_AUTOCOMMIT:
       assert ast.getChildCount() == 1;
@@ -12316,7 +12323,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     // 5. Resolve Parse Tree
     // Materialization is allowed if it is not a view definition
-    getMetaData(qb, createVwDesc == null);
+    getMetaData(qb, createVwDesc == null && !forViewCreation);
     LOG.info("Completed getting MetaData in Semantic Analysis");
 
     return true;
@@ -12447,7 +12454,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     ASTNode astForMasking;
     if (isCBOExecuted() && needsTransform &&
-        (qb.isCTAS() || qb.isView() || qb.isMaterializedView() || qb.isMultiDestQuery())) {
+        (qb.isCTAS() || forViewCreation || qb.isMaterializedView() || qb.isMultiDestQuery())) {
       // If we use CBO and we may apply masking/filtering policies, we create a copy of the ast.
       // The reason is that the generation of the operator tree may modify the initial ast,
       // but if we need to parse for a second time, we would like to parse the unmodified ast.
@@ -12457,7 +12464,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     // 2. Gen OP Tree from resolved Parse Tree
-    Operator sinkOp = genOPTree(ast, plannerCtx);
+    sinkOp = genOPTree(ast, plannerCtx);
 
     boolean usesMasking = false;
     if (!unparseTranslator.isEnabled() &&
@@ -12491,7 +12498,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     // 3. Deduce Resultset Schema
-    if (createVwDesc != null && !this.ctx.isCboSucceeded()) {
+    if ((forViewCreation || createVwDesc != null) && !this.ctx.isCboSucceeded()) {
       resultSchema = convertRowSchemaToViewSchema(opParseCtx.get(sinkOp).getRowResolver());
     } else {
       // resultSchema will be null if
@@ -12523,6 +12530,25 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     // Set the mapjoin hint if it needs to be disabled.
     pCtx.setDisableMapJoin(disableMapJoinWithHint(getQB().getParseInfo().getHintList()));
 
+    if (forViewCreation) {
+      // Generate lineage info if LineageLogger hook is configured.
+      // Add the transformation that computes the lineage information.
+      Set<String> postExecHooks = Sets.newHashSet(Splitter.on(",").trimResults()
+          .omitEmptyStrings()
+          .split(Strings.nullToEmpty(HiveConf.getVar(conf, HiveConf.ConfVars.POSTEXECHOOKS))));
+      if (postExecHooks.contains("org.apache.hadoop.hive.ql.hooks.PostExecutePrinter")
+          || postExecHooks.contains("org.apache.hadoop.hive.ql.hooks.LineageLogger")
+          || postExecHooks.contains("org.apache.atlas.hive.hook.HiveHook")) {
+        List<Transform> transformations = new ArrayList<Transform>();
+        transformations.add(new HiveOpConverterPostProc());
+        transformations.add(new Generator(postExecHooks));
+        for (Transform t : transformations) {
+          pCtx = t.transform(pCtx);
+        }
+      }
+      return;
+    }
+
     // 5. Take care of view creation
     if (createVwDesc != null) {
       if (ctx.getExplainAnalyze() == AnalyzeState.RUNNING) {
@@ -12537,42 +12563,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       // all the information for semanticcheck
       validateCreateView();
 
-      if (createVwDesc.isMaterialized()) {
-        createVwDesc.setTablesUsed(getTablesUsed(pCtx));
-      } else {
-        // Since we're only creating a view (not executing it), we don't need to
-        // optimize or translate the plan (and in fact, those procedures can
-        // interfere with the view creation). So skip the rest of this method.
-        ctx.setResDir(null);
-        ctx.setResFile(null);
-
-        try {
-          PlanUtils.addInputsForView(pCtx);
-        } catch (HiveException e) {
-          throw new SemanticException(e);
-        }
-
-        // Generate lineage info for create view statements
-        // if LineageLogger hook is configured.
-        // Add the transformation that computes the lineage information.
-        Set<String> postExecHooks = Sets.newHashSet(Splitter.on(",").trimResults()
-            .omitEmptyStrings()
-            .split(Strings.nullToEmpty(HiveConf.getVar(conf, HiveConf.ConfVars.POSTEXECHOOKS))));
-        if (postExecHooks.contains("org.apache.hadoop.hive.ql.hooks.PostExecutePrinter")
-            || postExecHooks.contains("org.apache.hadoop.hive.ql.hooks.LineageLogger")
-            || postExecHooks.contains("org.apache.atlas.hive.hook.HiveHook")) {
-          List<Transform> transformations = new ArrayList<Transform>();
-          transformations.add(new HiveOpConverterPostProc());
-          transformations.add(new Generator(postExecHooks));
-          for (Transform t : transformations) {
-            pCtx = t.transform(pCtx);
-          }
-          // we just use view name as location.
-          queryState.getLineageState()
-              .mapDirToOp(new Path(createVwDesc.getViewName()), sinkOp);
-        }
-        return;
-      }
+      createVwDesc.setTablesUsed(getTablesUsed(pCtx));
     }
 
     // 6. Generate table access stats if required
@@ -12691,12 +12682,11 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     return resultSchema;
   }
 
-  protected void saveViewDefinition() throws SemanticException {
-    if (createVwDesc.isMaterialized() && createVwDesc.isReplace()) {
-      // This is a rebuild, there's nothing to do here
-      return;
-    }
+  public List<FieldSchema> getOriginalResultSchema() {
+    return originalResultSchema;
+  }
 
+  protected void saveViewDefinition() throws SemanticException {
     // 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 =
@@ -12728,116 +12718,34 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     String expandedText = ctx.getTokenRewriteStream().toString(
         viewSelect.getTokenStartIndex(), viewSelect.getTokenStopIndex());
 
-    if (createVwDesc.isMaterialized()) {
-      if (createVwDesc.getPartColNames() != null) {
-        // If we are creating a materialized view and it has partition columns,
-        // we may need to reorder column projection in expanded query. The reason
-        // is that Hive assumes that in the partition columns are at the end of
-        // the MV schema, and if we do not do this, we will have a mismatch between
-        // the SQL query for the MV and the MV itself.
-        boolean first = true;
-        StringBuilder sb = new StringBuilder();
-        sb.append("SELECT ");
-        for (FieldSchema fieldSchema : derivedSchema) {
-          if (!createVwDesc.getPartColNames().contains(fieldSchema.getName())) {
-            if (first) {
-              first = false;
-            } else {
-              sb.append(", ");
-            }
-            sb.append(HiveUtils.unparseIdentifier(fieldSchema.getName(), conf));
-          }
-        }
-        for (String partColName : createVwDesc.getPartColNames()) {
-          sb.append(", ");
-          sb.append(HiveUtils.unparseIdentifier(partColName, conf));
-        }
-        sb.append(" FROM (");
-        sb.append(expandedText);
-        sb.append(") ");
-        sb.append(HiveUtils.unparseIdentifier(Utilities.getDbTableName(createVwDesc.getViewName())[1], conf));
-        expandedText = sb.toString();
-      }
-    } else {
-      if (imposedSchema != null) {
-        // Merge the names from the imposed schema into the types
-        // from the derived schema.
-        StringBuilder sb = new StringBuilder();
-        sb.append("SELECT ");
-        int n = derivedSchema.size();
-        for (int i = 0; i < n; ++i) {
-          if (i > 0) {
+    if (createVwDesc.getPartColNames() != null) {
+      // If we are creating a materialized view and it has partition columns,
+      // we may need to reorder column projection in expanded query. The reason
+      // is that Hive assumes that in the partition columns are at the end of
+      // the MV schema, and if we do not do this, we will have a mismatch between
+      // the SQL query for the MV and the MV itself.
+      boolean first = true;
+      StringBuilder sb = new StringBuilder();
+      sb.append("SELECT ");
+      for (FieldSchema fieldSchema : derivedSchema) {
+        if (!createVwDesc.getPartColNames().contains(fieldSchema.getName())) {
+          if (first) {
+            first = false;
+          } else {
             sb.append(", ");
           }
-          FieldSchema fieldSchema = derivedSchema.get(i);
-          // Modify a copy, not the original
-          fieldSchema = new FieldSchema(fieldSchema);
-          // TODO: there's a potential problem here if some table uses external schema like Avro,
-          //       with a very large type name. It seems like the view does not derive the SerDe from
-          //       the table, so it won't be able to just get the type from the deserializer like the
-          //       table does; we won't be able to properly store the type in the RDBMS metastore.
-          //       Not sure if these large cols could be in resultSchema. Ignore this for now 0_o
-          derivedSchema.set(i, fieldSchema);
           sb.append(HiveUtils.unparseIdentifier(fieldSchema.getName(), conf));
-          sb.append(" AS ");
-          String imposedName = imposedSchema.get(i).getName();
-          sb.append(HiveUtils.unparseIdentifier(imposedName, conf));
-          fieldSchema.setName(imposedName);
-          // We don't currently allow imposition of a type
-          fieldSchema.setComment(imposedSchema.get(i).getComment());
-        }
-        sb.append(" FROM (");
-        sb.append(expandedText);
-        sb.append(") ");
-        sb.append(HiveUtils.unparseIdentifier(Utilities.getDbTableName(createVwDesc.getViewName())[1], conf));
-        expandedText = sb.toString();
-      }
-
-      if (createVwDesc.getPartColNames() != null) {
-        // Make sure all partitioning columns referenced actually
-        // exist and are in the correct order at the end
-        // of the list of columns produced by the view. Also move the field
-        // schema descriptors from derivedSchema to the partitioning key
-        // descriptor.
-        List<String> partColNames = createVwDesc.getPartColNames();
-        if (partColNames.size() > derivedSchema.size()) {
-          throw new SemanticException(
-              ErrorMsg.VIEW_PARTITION_MISMATCH.getMsg());
-        }
-
-        // Get the partition columns from the end of derivedSchema.
-        List<FieldSchema> partitionColumns = derivedSchema.subList(
-            derivedSchema.size() - partColNames.size(),
-            derivedSchema.size());
-
-        // Verify that the names match the PARTITIONED ON clause.
-        Iterator<String> colNameIter = partColNames.iterator();
-        Iterator<FieldSchema> schemaIter = partitionColumns.iterator();
-        while (colNameIter.hasNext()) {
-          String colName = colNameIter.next();
-          FieldSchema fieldSchema = schemaIter.next();
-          if (!fieldSchema.getName().equals(colName)) {
-            throw new SemanticException(
-                ErrorMsg.VIEW_PARTITION_MISMATCH.getMsg());
-          }
-        }
-
-        // Boundary case: require at least one non-partitioned column
-        // for consistency with tables.
-        if (partColNames.size() == derivedSchema.size()) {
-          throw new SemanticException(
-              ErrorMsg.VIEW_PARTITION_TOTAL.getMsg());
         }
-
-        // Now make a copy.
-        createVwDesc.setPartCols(
-            new ArrayList<FieldSchema>(partitionColumns));
-
-        // Finally, remove the partition columns from the end of derivedSchema.
-        // (Clearing the subList writes through to the underlying
-        // derivedSchema ArrayList.)
-        partitionColumns.clear();
       }
+      for (String partColName : createVwDesc.getPartColNames()) {
+        sb.append(", ");
+        sb.append(HiveUtils.unparseIdentifier(partColName, conf));
+      }
+      sb.append(" FROM (");
+      sb.append(expandedText);
+      sb.append(") ");
+      sb.append(HiveUtils.unparseIdentifier(Utilities.getDbTableName(createVwDesc.getViewName())[1], conf));
+      expandedText = sb.toString();
     }
 
     // Set schema and expanded text for the view
@@ -13719,16 +13627,12 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     List<FieldSchema> cols = null;
     boolean ifNotExists = false;
     boolean rewriteEnabled = true;
-    boolean orReplace = false;
-    boolean isAlterViewAs = false;
     String comment = null;
     ASTNode selectStmt = null;
     Map<String, String> tblProps = null;
     List<String> partColNames = null;
     List<String> sortColNames = null;
     List<String> distributeColNames = null;
-    boolean isMaterialized = ast.getToken().getType() == HiveParser.TOK_CREATE_MATERIALIZED_VIEW;
-    boolean isRebuild = false;
     String location = null;
     RowFormatParams rowFormatParams = new RowFormatParams();
     StorageFormat storageFormat = new StorageFormat(conf);
@@ -13748,9 +13652,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       case HiveParser.TOK_REWRITE_DISABLED:
         rewriteEnabled = false;
         break;
-      case HiveParser.TOK_ORREPLACE:
-        orReplace = true;
-        break;
       case HiveParser.TOK_QUERY:
         // For CBO
         if (plannerCtx != null) {
@@ -13804,82 +13705,60 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       }
     }
 
-    storageFormat.fillDefaultStorageFormat(false, isMaterialized);
+    storageFormat.fillDefaultStorageFormat(false, true);
 
-    if (ifNotExists && orReplace) {
-      throw new SemanticException("Can't combine IF NOT EXISTS and OR REPLACE.");
-    }
-
-    if (isMaterialized) {
-      if (!ifNotExists) {
-        // Verify that the table does not already exist
-        // dumpTable is only used to check the conflict for non-temporary tables
-        try {
-          Table dumpTable = db.newTable(dbDotTable);
-          if (null != db.getTable(dumpTable.getDbName(), dumpTable.getTableName(), false) && !ctx.isExplainSkipExecution()) {
-            throw new SemanticException(ErrorMsg.TABLE_ALREADY_EXISTS.getMsg(dbDotTable));
-          }
-        } catch (HiveException e) {
-          throw new SemanticException(e);
+    if (!ifNotExists) {
+      // Verify that the table does not already exist
+      // dumpTable is only used to check the conflict for non-temporary tables
+      try {
+        Table dumpTable = db.newTable(dbDotTable);
+        if (null != db.getTable(dumpTable.getDbName(), dumpTable.getTableName(), false) &&
+            !ctx.isExplainSkipExecution()) {
+          throw new SemanticException(ErrorMsg.TABLE_ALREADY_EXISTS.getMsg(dbDotTable));
         }
+      } catch (HiveException e) {
+        throw new SemanticException(e);
       }
-      if (partColNames != null && (distributeColNames != null || sortColNames != null)) {
-        // Verify that partition columns and data organization columns are not overlapping
-        Set<String> partColNamesSet = new HashSet<>(partColNames);
-        if (distributeColNames != null) {
-          for (String colName : distributeColNames) {
-            if (partColNamesSet.contains(colName)) {
-              throw new SemanticException("Same column cannot be present in partition and cluster/distribute clause. "
-                  + "Column name: " + colName);
-            }
+    }
+    if (partColNames != null && (distributeColNames != null || sortColNames != null)) {
+      // Verify that partition columns and data organization columns are not overlapping
+      Set<String> partColNamesSet = new HashSet<>(partColNames);
+      if (distributeColNames != null) {
+        for (String colName : distributeColNames) {
+          if (partColNamesSet.contains(colName)) {
+            throw new SemanticException("Same column cannot be present in partition and cluster/distribute clause. "
+                + "Column name: " + colName);
           }
         }
-        if (sortColNames != null) {
-          for (String colName : sortColNames) {
-            if (partColNamesSet.contains(colName)) {
-              throw new SemanticException("Same column cannot be present in partition and cluster/sort clause. "
-                  + "Column name: " + colName);
-            }
+      }
+      if (sortColNames != null) {
+        for (String colName : sortColNames) {
+          if (partColNamesSet.contains(colName)) {
+            throw new SemanticException("Same column cannot be present in partition and cluster/sort clause. "
+                + "Column name: " + colName);
           }
         }
       }
     }
 
-    if (ast.getToken().getType() == HiveParser.TOK_ALTERVIEW &&
-        ast.getChild(1).getType() == HiveParser.TOK_QUERY) {
-      isAlterViewAs = true;
-      orReplace = true;
-    }
-
     unparseTranslator.enable();
 
-    if (isMaterialized) {
-      if (makeAcid()) {
-        if (tblProps == null) {
-          tblProps = new HashMap<>();
-        }
-        tblProps = convertToAcidByDefault(storageFormat, dbDotTable, null, tblProps);
+    if (makeAcid()) {
+      if (tblProps == null) {
+        tblProps = new HashMap<>();
       }
-
-      createVwDesc = new CreateViewDesc(
-          dbDotTable, cols, comment, tblProps, partColNames, sortColNames, distributeColNames,
-          ifNotExists, isRebuild, rewriteEnabled, isAlterViewAs,
-          storageFormat.getInputFormat(), storageFormat.getOutputFormat(),
-          location, storageFormat.getSerde(), storageFormat.getStorageHandler(),
-          storageFormat.getSerdeProps());
-      addDbAndTabToOutputs(new String[] {qualTabName.getDb(), qualTabName.getTable()}, TableType.MATERIALIZED_VIEW,
-          false, tblProps);
-      queryState.setCommandType(HiveOperation.CREATE_MATERIALIZED_VIEW);
-    } else {
-      createVwDesc = new CreateViewDesc(
-          dbDotTable, cols, comment, tblProps, partColNames,
-          ifNotExists, orReplace, isAlterViewAs, storageFormat.getInputFormat(),
-          storageFormat.getOutputFormat(), storageFormat.getSerde());
-      rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), createVwDesc)));
-      addDbAndTabToOutputs(new String[] {qualTabName.getDb(), qualTabName.getTable()},
-          TableType.VIRTUAL_VIEW, false, tblProps);
-      queryState.setCommandType(HiveOperation.CREATEVIEW);
+      tblProps = convertToAcidByDefault(storageFormat, dbDotTable, null, tblProps);
     }
+
+    createVwDesc = new CreateMaterializedViewDesc(
+        dbDotTable, cols, comment, tblProps, partColNames, sortColNames, distributeColNames,
+        ifNotExists, rewriteEnabled,
+        storageFormat.getInputFormat(), storageFormat.getOutputFormat(),
+        location, storageFormat.getSerde(), storageFormat.getStorageHandler(),
+        storageFormat.getSerdeProps());
+    addDbAndTabToOutputs(new String[] {qualTabName.getDb(), qualTabName.getTable()}, TableType.MATERIALIZED_VIEW,
+        false, tblProps);
+    queryState.setCommandType(HiveOperation.CREATE_MATERIALIZED_VIEW);
     qb.setViewDesc(createVwDesc);
 
     return selectStmt;
@@ -13896,8 +13775,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   private void validateCreateView()
       throws SemanticException {
     try {
-      Table oldView = getTable(createVwDesc.getViewName(), false);
-
       // Do not allow view to be defined on temp table or other materialized view
       Set<String> tableAliases = qb.getTabAliases();
       for (String alias : tableAliases) {
@@ -13912,8 +13789,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           if (table.isMaterializedView()) {
             throw new SemanticException("View definition references materialized view " + alias);
           }
-          if (createVwDesc.isMaterialized() && createVwDesc.isRewriteEnabled() &&
-              !AcidUtils.isTransactionalTable(table)) {
+          if (createVwDesc.isRewriteEnabled() && !AcidUtils.isTransactionalTable(table)) {
             throw new SemanticException("Automatic rewriting for materialized view cannot "
                 + "be enabled if the materialized view uses non-transactional tables");
           }
@@ -13922,11 +13798,11 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         }
       }
 
-      if (createVwDesc.isMaterialized() && !qb.hasTableDefined()) {
+      if (!qb.hasTableDefined()) {
         throw new SemanticException("Materialized view must have a table defined.");
       }
 
-      if (createVwDesc.isMaterialized() && createVwDesc.isRewriteEnabled()) {
+      if (createVwDesc.isRewriteEnabled()) {
         if (!ctx.isCboSucceeded()) {
           String msg = "Cannot enable automatic rewriting for materialized view.";
           if (ctx.getCboInfo() != null) {
@@ -13939,60 +13815,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
               getInvalidAutomaticRewritingMaterializationReason());
         }
       }
-
-      // ALTER VIEW AS SELECT requires the view must exist
-      if (createVwDesc.getIsAlterViewAs() && oldView == null) {
-        String viewNotExistErrorMsg =
-            "The following view does not exist: " + createVwDesc.getViewName();
-        throw new SemanticException(
-            ErrorMsg.ALTER_VIEW_AS_SELECT_NOT_EXIST.getMsg(viewNotExistErrorMsg));
-      }
-
-      //replace view
-      if (createVwDesc.isReplace() && oldView != null) {
-
-        // Don't allow swapping between virtual and materialized view in replace
-        if (oldView.getTableType().equals(TableType.VIRTUAL_VIEW) && createVwDesc.isMaterialized()) {
-          throw new SemanticException(ErrorMsg.REPLACE_VIEW_WITH_MATERIALIZED,
-              oldView.getTableName());
-        }
-
-        if (oldView.getTableType().equals(TableType.MATERIALIZED_VIEW) &&
-            !createVwDesc.isMaterialized()) {
-          throw new SemanticException(ErrorMsg.REPLACE_MATERIALIZED_WITH_VIEW,
-              oldView.getTableName());
-        }
-
-        // Existing table is not a view
-        if (!oldView.getTableType().equals(TableType.VIRTUAL_VIEW) &&
-            !oldView.getTableType().equals(TableType.MATERIALIZED_VIEW)) {
-          String tableNotViewErrorMsg =
-              "The following is an existing table, not a view: " +
-                  createVwDesc.getViewName();
-          throw new SemanticException(
-              ErrorMsg.EXISTING_TABLE_IS_NOT_VIEW.getMsg(tableNotViewErrorMsg));
-        }
-
-        if (!createVwDesc.isMaterialized()) {
-          // if old view has partitions, it could not be replaced
-          String partitionViewErrorMsg =
-              "The following view has partition, it could not be replaced: " +
-                  createVwDesc.getViewName();
-          try {
-            if ((createVwDesc.getPartCols() == null ||
-                createVwDesc.getPartCols().isEmpty() ||
-                !createVwDesc.getPartCols().equals(oldView.getPartCols())) &&
-                !oldView.getPartCols().isEmpty() &&
-                !db.getPartitions(oldView).isEmpty()) {
-              throw new SemanticException(
-                  ErrorMsg.REPLACE_VIEW_WITH_PARTITION.getMsg(partitionViewErrorMsg));
-            }
-          } catch (HiveException e) {
-            throw new SemanticException(
-                ErrorMsg.REPLACE_VIEW_WITH_PARTITION.getMsg(partitionViewErrorMsg));
-          }
-        }
-      }
     } catch (HiveException e) {
       throw new SemanticException(e.getMessage(), e);
     }
@@ -14943,7 +14765,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       queryProperties.setHasOuterOrderBy(!qb.getParseInfo().getIsSubQ() &&
           !qb.getParseInfo().getDestToOrderBy().isEmpty());
       queryProperties.setOuterQueryLimit(qb.getParseInfo().getOuterQueryLimit());
-      queryProperties.setMaterializedView(qb.getViewDesc() != null);
+      queryProperties.setMaterializedView(qb.isMaterializedView());
     }
   }
 
@@ -15281,6 +15103,23 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     return (invalidResultCacheReason == null);
   }
 
+  public void forViewCreation(String fqViewName) {
+    this.fqViewName = fqViewName;
+    this.forViewCreation = true;
+  }
+
+  public Map<String, TableScanOperator> getTopOps() {
+    return topOps;
+  }
+
+  public Map<String, ReadEntity> getViewAliasToInput() {
+    return viewAliasToInput;
+  }
+
+  public Operator getSinkOp() {
+    return sinkOp;
+  }
+
   protected enum MaterializationRebuildMode {
     NONE,
     INSERT_OVERWRITE_REBUILD,
@@ -15354,7 +15193,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   }
 
   @Override
-  protected void executeUnparseTranlations() {
+  public void executeUnparseTranlations() {
     unparseTranslator.applyTranslations(ctx.getTokenRewriteStream());
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java
index 2350646..b9f283f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java
@@ -104,7 +104,7 @@ public class StorageFormat {
     }
   }
 
-  protected void fillDefaultStorageFormat(boolean isExternal, boolean isMaterializedView)
+  public void fillDefaultStorageFormat(boolean isExternal, boolean isMaterializedView)
       throws  SemanticException {
     if ((inputFormat == null) && (storageHandler == null)) {
       String defaultFormat;
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 b3d0f12..201d676 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
@@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.ddl.DDLDesc;
 import org.apache.hadoop.hive.ql.ddl.DDLTask;
 import org.apache.hadoop.hive.ql.ddl.DDLWork;
 import org.apache.hadoop.hive.ql.ddl.table.create.CreateTableDesc;
-import org.apache.hadoop.hive.ql.ddl.view.create.CreateViewDesc;
+import org.apache.hadoop.hive.ql.ddl.view.create.CreateMaterializedViewDesc;
 import org.apache.hadoop.hive.ql.ddl.view.materialized.alter.rewrite.AlterMaterializedViewRewriteDesc;
 import org.apache.hadoop.hive.ql.ddl.view.materialized.update.MaterializedViewUpdateDesc;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
@@ -366,7 +366,7 @@ public abstract class TaskCompiler {
           CollectionUtils.isEmpty(crtTblDesc.getPartColNames()));
     } else if (pCtx.getQueryProperties().isMaterializedView()) {
       // generate a DDL task and make it a dependent task of the leaf
-      CreateViewDesc viewDesc = pCtx.getCreateViewDesc();
+      CreateMaterializedViewDesc viewDesc = pCtx.getCreateViewDesc();
       Task<?> crtViewTask = TaskFactory.get(new DDLWork(
           inputs, outputs, viewDesc));
       patchUpAfterCTASorMaterializedView(rootTasks, inputs, outputs, crtViewTask,
@@ -454,7 +454,7 @@ public abstract class TaskCompiler {
       txnId = ctd.getInitialMmWriteId();
       loc = ctd.getLocation();
     } else {
-      CreateViewDesc cmv = pCtx.getCreateViewDesc();
+      CreateMaterializedViewDesc cmv = pCtx.getCreateViewDesc();
       dataSink = cmv.getAndUnsetWriter();
       txnId = cmv.getInitialMmWriteId();
       loc = cmv.getLocation();
@@ -555,16 +555,14 @@ public abstract class TaskCompiler {
       DDLTask ddlTask = (DDLTask)createTask;
       DDLWork work = ddlTask.getWork();
       DDLDesc desc = work.getDDLDesc();
-      if (desc instanceof CreateViewDesc) {
-        CreateViewDesc createViewDesc = (CreateViewDesc)desc;
-        if (createViewDesc.isMaterialized()) {
-          String tableName = createViewDesc.getViewName();
-          boolean retrieveAndInclude = createViewDesc.isRewriteEnabled();
-          MaterializedViewUpdateDesc materializedViewUpdateDesc =
-              new MaterializedViewUpdateDesc(tableName, retrieveAndInclude, false, false);
-          DDLWork ddlWork = new DDLWork(inputs, outputs, materializedViewUpdateDesc);
-          targetTask.addDependentTask(TaskFactory.get(ddlWork, conf));
-        }
+      if (desc instanceof CreateMaterializedViewDesc) {
+        CreateMaterializedViewDesc createViewDesc = (CreateMaterializedViewDesc)desc;
+        String tableName = createViewDesc.getViewName();
+        boolean retrieveAndInclude = createViewDesc.isRewriteEnabled();
+        MaterializedViewUpdateDesc materializedViewUpdateDesc =
+            new MaterializedViewUpdateDesc(tableName, retrieveAndInclude, false, false);
+        DDLWork ddlWork = new DDLWork(inputs, outputs, materializedViewUpdateDesc);
+        targetTask.addDependentTask(TaskFactory.get(ddlWork, conf));
       } else if (desc instanceof AlterMaterializedViewRewriteDesc) {
         AlterMaterializedViewRewriteDesc alterMVRewriteDesc = (AlterMaterializedViewRewriteDesc)desc;
         String tableName = alterMVRewriteDesc.getMaterializedViewName();
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 503f731..6534f0d 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
@@ -111,8 +111,8 @@ public enum HiveOperation {
   RELOADFUNCTION("RELOADFUNCTION", HiveParser.TOK_RELOADFUNCTIONS, null, null),
   CREATEMACRO("CREATEMACRO", HiveParser.TOK_CREATEMACRO, null, null),
   DROPMACRO("DROPMACRO", HiveParser.TOK_DROPMACRO, null, null),
-  CREATEVIEW("CREATEVIEW", HiveParser.TOK_CREATEVIEW, new Privilege[]{Privilege.SELECT},
-      new Privilege[]{Privilege.CREATE}),
+  CREATEVIEW("CREATEVIEW", new int[] {HiveParser.TOK_CREATEVIEW, HiveParser.TOK_ALTERVIEW_AS},
+      new Privilege[]{Privilege.SELECT}, new Privilege[]{Privilege.CREATE}),
   CREATE_MATERIALIZED_VIEW("CREATE_MATERIALIZED_VIEW", HiveParser.TOK_CREATE_MATERIALIZED_VIEW,
       new Privilege[]{Privilege.SELECT}, new Privilege[]{Privilege.CREATE}),
   DROPVIEW("DROPVIEW", HiveParser.TOK_DROPVIEW, null, new Privilege[]{Privilege.DROP}),
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/LoadFileDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/LoadFileDesc.java
index 07bcef8..d9246b3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/LoadFileDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/LoadFileDesc.java
@@ -22,7 +22,7 @@ import java.io.Serializable;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.ddl.table.create.CreateTableDesc;
-import org.apache.hadoop.hive.ql.ddl.view.create.CreateViewDesc;
+import org.apache.hadoop.hive.ql.ddl.view.create.CreateMaterializedViewDesc;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 
@@ -38,7 +38,7 @@ public class LoadFileDesc extends LoadDesc implements Serializable {
   private String columns;
   private String columnTypes;
   private transient CreateTableDesc ctasCreateTableDesc;
-  private transient CreateViewDesc createViewDesc;
+  private transient CreateMaterializedViewDesc createViewDesc;
   private boolean isMmCtas;
 
   public LoadFileDesc(final LoadFileDesc o) {
@@ -53,14 +53,14 @@ public class LoadFileDesc extends LoadDesc implements Serializable {
     this.createViewDesc = o.createViewDesc;
   }
 
-  public LoadFileDesc(final CreateTableDesc createTableDesc, final CreateViewDesc createViewDesc,
+  public LoadFileDesc(final CreateTableDesc createTableDesc, final CreateMaterializedViewDesc createViewDesc,
       final Path sourcePath, final Path targetDir, final boolean isDfsDir,
       final String columns, final String columnTypes, AcidUtils.Operation writeType, boolean isMmCtas) {
     this(sourcePath, targetDir, isDfsDir, columns, columnTypes, writeType, isMmCtas);
     if (createTableDesc != null && createTableDesc.isCTAS()) {
       this.ctasCreateTableDesc = createTableDesc;
     }
-    if (createViewDesc != null && createViewDesc.isMaterialized()) {
+    if (createViewDesc != null) {
       this.createViewDesc = createViewDesc;
     }
   }
@@ -136,7 +136,7 @@ public class LoadFileDesc extends LoadDesc implements Serializable {
     return ctasCreateTableDesc;
   }
 
-  public CreateViewDesc getCreateViewDesc() {
+  public CreateMaterializedViewDesc getCreateViewDesc() {
     return createViewDesc;
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
index fd3918a..996b2db 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.ddl.table.create.CreateTableDesc;
-import org.apache.hadoop.hive.ql.ddl.view.create.CreateViewDesc;
+import org.apache.hadoop.hive.ql.ddl.view.create.CreateMaterializedViewDesc;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
@@ -410,7 +410,7 @@ public final class PlanUtils {
  /**
    * Generate a table descriptor from a createViewDesc.
    */
-  public static TableDesc getTableDesc(CreateViewDesc crtViewDesc, String cols, String colTypes) {
+  public static TableDesc getTableDesc(CreateMaterializedViewDesc crtViewDesc, String cols, String colTypes) {
     TableDesc ret;
 
     try {
@@ -444,7 +444,7 @@ public final class PlanUtils {
                 crtViewDesc.getStorageHandler());
       }
 
-      if (crtViewDesc.getViewName() != null && crtViewDesc.isMaterialized()) {
+      if (crtViewDesc.getViewName() != null) {
         properties.setProperty(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_NAME,
             crtViewDesc.getViewName());
       }
diff --git a/ql/src/test/results/clientnegative/create_or_replace_view4.q.out b/ql/src/test/results/clientnegative/create_or_replace_view4.q.out
index 767cc77..b83939f 100644
--- a/ql/src/test/results/clientnegative/create_or_replace_view4.q.out
+++ b/ql/src/test/results/clientnegative/create_or_replace_view4.q.out
@@ -14,4 +14,4 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
 POSTHOOK: Lineage: v.key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: v.value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
-FAILED: SemanticException org.apache.hadoop.hive.ql.optimizer.calcite.CalciteViewSemanticException: At least one non-partitioning column must be present in view
+FAILED: SemanticException [Error 10092]: At least one non-partitioning column must be present in view
diff --git a/ql/src/test/results/clientnegative/create_view_failure10.q.out b/ql/src/test/results/clientnegative/create_view_failure10.q.out
index 62557cb..5aedfd3 100644
--- a/ql/src/test/results/clientnegative/create_view_failure10.q.out
+++ b/ql/src/test/results/clientnegative/create_view_failure10.q.out
@@ -6,4 +6,4 @@ POSTHOOK: query: create temporary table tmp1 (c1 string, c2 string)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@tmp1
-FAILED: SemanticException org.apache.hadoop.hive.ql.parse.SemanticException: View definition references temporary table tmp1
+FAILED: SemanticException View definition references temporary table tmp1
diff --git a/ql/src/test/results/clientnegative/create_view_failure3.q.out b/ql/src/test/results/clientnegative/create_view_failure3.q.out
index 8b79272..5ddbdb6 100644
--- a/ql/src/test/results/clientnegative/create_view_failure3.q.out
+++ b/ql/src/test/results/clientnegative/create_view_failure3.q.out
@@ -2,4 +2,4 @@ PREHOOK: query: DROP VIEW xxx13
 PREHOOK: type: DROPVIEW
 POSTHOOK: query: DROP VIEW xxx13
 POSTHOOK: type: DROPVIEW
-FAILED: SemanticException org.apache.hadoop.hive.ql.optimizer.calcite.CalciteViewSemanticException: 5:16 The number of columns produced by the SELECT clause does not match the number of column names specified by CREATE VIEW. Error encountered near token 'key'
+FAILED: SemanticException 5:16 The number of columns produced by the SELECT clause does not match the number of column names specified by CREATE VIEW. Error encountered near token 'key'
diff --git a/ql/src/test/results/clientnegative/create_view_failure5.q.out b/ql/src/test/results/clientnegative/create_view_failure5.q.out
index b7b3984..d79dc64 100644
--- a/ql/src/test/results/clientnegative/create_view_failure5.q.out
+++ b/ql/src/test/results/clientnegative/create_view_failure5.q.out
@@ -2,4 +2,4 @@ PREHOOK: query: DROP VIEW xxx14
 PREHOOK: type: DROPVIEW
 POSTHOOK: query: DROP VIEW xxx14
 POSTHOOK: type: DROPVIEW
-FAILED: SemanticException org.apache.hadoop.hive.ql.optimizer.calcite.CalciteViewSemanticException: Duplicate column name: key
+FAILED: SemanticException [Error 10036]: Duplicate column name: key
diff --git a/ql/src/test/results/clientnegative/create_view_failure6.q.out b/ql/src/test/results/clientnegative/create_view_failure6.q.out
index 6d9fb64..25c1c7f 100644
--- a/ql/src/test/results/clientnegative/create_view_failure6.q.out
+++ b/ql/src/test/results/clientnegative/create_view_failure6.q.out
@@ -2,4 +2,4 @@ PREHOOK: query: DROP VIEW xxx15
 PREHOOK: type: DROPVIEW
 POSTHOOK: query: DROP VIEW xxx15
 POSTHOOK: type: DROPVIEW
-FAILED: SemanticException org.apache.hadoop.hive.ql.optimizer.calcite.CalciteViewSemanticException: Rightmost columns in view output do not match PARTITIONED ON clause
+FAILED: SemanticException [Error 10093]: Rightmost columns in view output do not match PARTITIONED ON clause
diff --git a/ql/src/test/results/clientnegative/create_view_failure7.q.out b/ql/src/test/results/clientnegative/create_view_failure7.q.out
index 337dbe8..f13ab63 100644
--- a/ql/src/test/results/clientnegative/create_view_failure7.q.out
+++ b/ql/src/test/results/clientnegative/create_view_failure7.q.out
@@ -2,4 +2,4 @@ PREHOOK: query: DROP VIEW xxx16
 PREHOOK: type: DROPVIEW
 POSTHOOK: query: DROP VIEW xxx16
 POSTHOOK: type: DROPVIEW
-FAILED: SemanticException org.apache.hadoop.hive.ql.optimizer.calcite.CalciteViewSemanticException: At least one non-partitioning column must be present in view
+FAILED: SemanticException [Error 10092]: At least one non-partitioning column must be present in view
diff --git a/ql/src/test/results/clientnegative/create_view_failure8.q.out b/ql/src/test/results/clientnegative/create_view_failure8.q.out
index cccb7e4..158fed1 100644
--- a/ql/src/test/results/clientnegative/create_view_failure8.q.out
+++ b/ql/src/test/results/clientnegative/create_view_failure8.q.out
@@ -2,4 +2,4 @@ PREHOOK: query: DROP VIEW xxx17
 PREHOOK: type: DROPVIEW
 POSTHOOK: query: DROP VIEW xxx17
 POSTHOOK: type: DROPVIEW
-FAILED: SemanticException org.apache.hadoop.hive.ql.optimizer.calcite.CalciteViewSemanticException: Rightmost columns in view output do not match PARTITIONED ON clause
+FAILED: SemanticException [Error 10093]: Rightmost columns in view output do not match PARTITIONED ON clause
diff --git a/ql/src/test/results/clientnegative/create_view_failure9.q.out b/ql/src/test/results/clientnegative/create_view_failure9.q.out
index eac8cb4..e6ad388 100644
--- a/ql/src/test/results/clientnegative/create_view_failure9.q.out
+++ b/ql/src/test/results/clientnegative/create_view_failure9.q.out
@@ -2,4 +2,4 @@ PREHOOK: query: DROP VIEW xxx18
 PREHOOK: type: DROPVIEW
 POSTHOOK: query: DROP VIEW xxx18
 POSTHOOK: type: DROPVIEW
-FAILED: SemanticException org.apache.hadoop.hive.ql.optimizer.calcite.CalciteViewSemanticException: Rightmost columns in view output do not match PARTITIONED ON clause
+FAILED: SemanticException [Error 10093]: Rightmost columns in view output do not match PARTITIONED ON clause
diff --git a/ql/src/test/results/clientnegative/masking_mv.q.out b/ql/src/test/results/clientnegative/masking_mv.q.out
index 926c506..801be5d 100644
--- a/ql/src/test/results/clientnegative/masking_mv.q.out
+++ b/ql/src/test/results/clientnegative/masking_mv.q.out
@@ -106,7 +106,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
 
   Stage: Stage-8
-    Create View
+    Create Materialized View
       columns: key int
       expanded text: select `masking_test_n_mv`.`key` from `default`.`masking_test_n_mv`
       name: default.masking_test_view_n_mv
diff --git a/ql/src/test/results/clientnegative/selectDistinctStarNeg_1.q.out b/ql/src/test/results/clientnegative/selectDistinctStarNeg_1.q.out
index 9496e52..2d97755 100644
--- a/ql/src/test/results/clientnegative/selectDistinctStarNeg_1.q.out
+++ b/ql/src/test/results/clientnegative/selectDistinctStarNeg_1.q.out
@@ -2,4 +2,4 @@ PREHOOK: query: drop view if exists v
 PREHOOK: type: DROPVIEW
 POSTHOOK: query: drop view if exists v
 POSTHOOK: type: DROPVIEW
-FAILED: SemanticException org.apache.hadoop.hive.ql.optimizer.calcite.CalciteViewSemanticException: Duplicate column name: key
+FAILED: SemanticException [Error 10036]: Duplicate column name: key
diff --git a/ql/src/test/results/clientpositive/llap/create_view.q.out b/ql/src/test/results/clientpositive/llap/create_view.q.out
index 52b77c7..fe03404 100644
--- a/ql/src/test/results/clientpositive/llap/create_view.q.out
+++ b/ql/src/test/results/clientpositive/llap/create_view.q.out
@@ -168,15 +168,15 @@ POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@view0
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
+  Stage: Stage-0
     Create View
-      columns: valoo string
       expanded text: SELECT `_c0` AS `valoo` FROM (SELECT upper(`src`.`value`) FROM `default`.`src` WHERE `src`.`key`=86) `view0`
-      name: default.view0
       original text: SELECT upper(value) FROM src WHERE key=86
+      columns: valoo string
+      name: default.view0
 
 PREHOOK: query: EXPLAIN
 SELECT * from view2 where key=18
diff --git a/ql/src/test/results/clientpositive/llap/create_view_translate.q.out b/ql/src/test/results/clientpositive/llap/create_view_translate.q.out
index b5d464e..94bf1bd 100644
--- a/ql/src/test/results/clientpositive/llap/create_view_translate.q.out
+++ b/ql/src/test/results/clientpositive/llap/create_view_translate.q.out
@@ -135,15 +135,15 @@ POSTHOOK: Input: default@items
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@priceview
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
+  Stage: Stage-0
     Create View
-      columns: id int, _c1 string
       expanded text: SELECT `items`.`id`, `items`.`info`['price'] FROM `default`.`items`
-      name: default.priceview
       original text: SELECT items.id, items.info['price'] FROM items
+      columns: id int, _c1 string
+      name: default.priceview
 
 PREHOOK: query: CREATE VIEW priceview AS SELECT items.id, items.info['price'] FROM items
 PREHOOK: type: CREATEVIEW
diff --git a/ql/src/test/results/clientpositive/llap/explain_ddl.q.out b/ql/src/test/results/clientpositive/llap/explain_ddl.q.out
index 3cada1b..974bfd1 100644
--- a/ql/src/test/results/clientpositive/llap/explain_ddl.q.out
+++ b/ql/src/test/results/clientpositive/llap/explain_ddl.q.out
@@ -473,15 +473,15 @@ POSTHOOK: Input: default@m1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@V1_n0
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
+  Stage: Stage-0
     Create View
-      columns: key string, value string
       expanded text: select `m1`.`key`, `m1`.`value` from `default`.`M1`
-      name: default.V1_n0
       original text: select * from M1
+      columns: key string, value string
+      name: default.V1_n0
 
 PREHOOK: query: EXPLAIN CREATE TABLE M1 LIKE src
 PREHOOK: type: CREATETABLE
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
index eaed0bc..e26de03 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
@@ -5468,8 +5468,8 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@mfgr_price_view_n3
 Plan optimized by CBO.
 
-Stage-1
-  Create View{"name:":"default.mfgr_price_view_n3","original text:":"select p_mfgr, p_brand, \nsum(p_retailprice) as s \nfrom part \ngroup by p_mfgr, p_brand"}
+Stage-0
+  Create View{"original text:":"select p_mfgr, p_brand, \nsum(p_retailprice) as s \nfrom part \ngroup by p_mfgr, p_brand","name:":"default.mfgr_price_view_n3"}
 
 PREHOOK: query: CREATE TABLE part_4_n1( 
 p_mfgr STRING, 
diff --git a/ql/src/test/results/clientpositive/llap/lineage3.q.out b/ql/src/test/results/clientpositive/llap/lineage3.q.out
index c00acb9..964075f 100644
--- a/ql/src/test/results/clientpositive/llap/lineage3.q.out
+++ b/ql/src/test/results/clientpositive/llap/lineage3.q.out
@@ -282,7 +282,7 @@ PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest_v2
-{"version":"1.0","engine":"tez","database":"default","hash":"3c9d233e2b71f8bfe0a511ffe53921d2","queryText":"create view dest_v2 (a, b) as select c1, x2\nfrom (\n  select c1, min(c2) x2\n  from (\n    select c1, c2, c3\n    from (\n      select cint c1, ctinyint c2, min(cfloat) c3\n      from alltypesorc\n      group by cint, ctinyint\n      order by cint, ctinyint\n      limit 1\n    ) x\n  ) x2\n  group by c1\n) y\norder by x2,c1 desc","edges":[{"sources":[2],"targets":[0],"edgeType":"P [...]
+{"version":"1.0","engine":"tez","database":"default","hash":"3c9d233e2b71f8bfe0a511ffe53921d2","queryText":"create view dest_v2 (a, b) as select c1, x2\nfrom (\n  select c1, min(c2) x2\n  from (\n    select c1, c2, c3\n    from (\n      select cint c1, ctinyint c2, min(cfloat) c3\n      from alltypesorc\n      group by cint, ctinyint\n      order by cint, ctinyint\n      limit 1\n    ) x\n  ) x2\n  group by c1\n) y\norder by x2,c1 desc","edges":[{"sources":[2],"targets":[0],"edgeType":"P [...]
 PREHOOK: query: drop view if exists dest_v3
 PREHOOK: type: DROPVIEW
 PREHOOK: query: create view dest_v3 (a1, a2, a3, a4, a5, a6, a7) as
@@ -303,7 +303,7 @@ PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest_v3
-{"version":"1.0","engine":"tez","database":"default","hash":"9848a9a38a4f6f031dc669e7e495f9ee","queryText":"create view dest_v3 (a1, a2, a3, a4, a5, a6, a7) as\n  select x.csmallint, x.cbigint bint1, x.ctinyint, c.cbigint bint2, x.cint, x.cfloat, c.cstring1\n  from alltypesorc c\n  join (\n     select a.csmallint csmallint, a.ctinyint ctinyint, a.cstring2 cstring2,\n           a.cint cint, a.cstring1 ctring1, b.cfloat cfloat, b.cbigint cbigint\n     from ( select * from alltypesorc a whe [...]
+{"version":"1.0","engine":"tez","database":"default","hash":"9848a9a38a4f6f031dc669e7e495f9ee","queryText":"create view dest_v3 (a1, a2, a3, a4, a5, a6, a7) as\n  select x.csmallint, x.cbigint bint1, x.ctinyint, c.cbigint bint2, x.cint, x.cfloat, c.cstring1\n  from alltypesorc c\n  join (\n     select a.csmallint csmallint, a.ctinyint ctinyint, a.cstring2 cstring2,\n           a.cint cint, a.cstring1 ctring1, b.cfloat cfloat, b.cbigint cbigint\n     from ( select * from alltypesorc a whe [...]
 PREHOOK: query: alter view dest_v3 as
   select * from (
     select sum(a.ctinyint) over (partition by a.csmallint order by a.csmallint) a,
@@ -317,7 +317,7 @@ PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest_v3
-{"version":"1.0","engine":"tez","database":"default","hash":"81bb549360513aeae39a3bd971405be3","queryText":"alter view dest_v3 as\n  select * from (\n    select sum(a.ctinyint) over (partition by a.csmallint order by a.csmallint) a,\n      count(b.cstring1) x, b.cboolean1\n    from alltypesorc a join alltypesorc b on (a.cint = b.cint)\n    where a.cboolean2 = true and b.cfloat > 0\n    group by a.ctinyint, a.csmallint, b.cboolean1\n    having count(a.cint) > 10\n    order by a, x, b.cboo [...]
+{"version":"1.0","engine":"tez","database":"default","hash":"81bb549360513aeae39a3bd971405be3","queryText":"alter view dest_v3 as\n  select * from (\n    select sum(a.ctinyint) over (partition by a.csmallint order by a.csmallint) a,\n      count(b.cstring1) x, b.cboolean1\n    from alltypesorc a join alltypesorc b on (a.cint = b.cint)\n    where a.cboolean2 = true and b.cfloat > 0\n    group by a.ctinyint, a.csmallint, b.cboolean1\n    having count(a.cint) > 10\n    order by a, x, b.cboo [...]
 PREHOOK: query: select * from dest_v3 limit 2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
diff --git a/ql/src/test/results/clientpositive/llap/masking_mv.q.out b/ql/src/test/results/clientpositive/llap/masking_mv.q.out
index 196688a..b0168dc 100644
--- a/ql/src/test/results/clientpositive/llap/masking_mv.q.out
+++ b/ql/src/test/results/clientpositive/llap/masking_mv.q.out
@@ -100,7 +100,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       columns: key int
       expanded text: select `masking_test_n_mv`.`key` from `default`.`masking_test_n_mv`
       name: default.masking_test_view_n_mv
@@ -793,7 +793,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       columns: key int
       expanded text: select `srctnx`.`key` from `default`.`srcTnx`
       name: default.masking_test_view_n_mv_2
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_cluster.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_cluster.q.out
index e34147d..37d590c 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_cluster.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_cluster.q.out
@@ -119,7 +119,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       distribute columns: key string
       columns: value string, key string
       sort columns: key string
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_3.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_3.q.out
index 6e6ee34..b77e47d 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_3.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_3.q.out
@@ -213,7 +213,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       columns: a int, c decimal(10,2)
       expanded text: SELECT `cmv_basetable`.`a`, `cmv_basetable_2`.`c`
   FROM `default`.`cmv_basetable` JOIN `default`.`cmv_basetable_2` ON (`cmv_basetable`.`a` = `cmv_basetable_2`.`a`)
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_4.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_4.q.out
index 2348924..6a3a159 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_4.q.out
@@ -217,7 +217,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       columns: a int, c decimal(10,2), _c2 bigint
       table properties:
         transactional true
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_rebuild_dummy.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_rebuild_dummy.q.out
index d4bf2a6..82881a0 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_rebuild_dummy.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_rebuild_dummy.q.out
@@ -213,7 +213,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       columns: a int, c decimal(10,2)
       expanded text: SELECT `cmv_basetable_n1`.`a`, `cmv_basetable_2_n0`.`c`
   FROM `default`.`cmv_basetable_n1` JOIN `default`.`cmv_basetable_2_n0` ON (`cmv_basetable_n1`.`a` = `cmv_basetable_2_n0`.`a`)
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window.q.out
index dc0e861..95c0ad3 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window.q.out
@@ -213,7 +213,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       columns: a int, c decimal(10,2)
       table properties:
         rewriting.time.window 5min
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_distribute_sort.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_distribute_sort.q.out
index cdcc356..c9d6709 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_distribute_sort.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_distribute_sort.q.out
@@ -119,7 +119,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       distribute columns: key string
       columns: value string, key string
       sort columns: value string
@@ -401,7 +401,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       distribute columns: value string
       columns: key string, value string
       sort columns: value string, key string
@@ -935,7 +935,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       distribute columns: key string
       columns: value string, key string
       sort columns: value string, key string
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_partition_cluster.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_partition_cluster.q.out
index 831953c..b89f928 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_partition_cluster.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_partition_cluster.q.out
@@ -125,7 +125,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       distribute columns: key string
       partition columns: partkey double
       columns: value string, key string
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_partitioned.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_partitioned.q.out
index 8951f00..5e7880d 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_partitioned.q.out
@@ -125,7 +125,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       partition columns: key string
       columns: value string
       expanded text: SELECT `value`, `key` FROM (SELECT `src_txn`.`value`, `src_txn`.`key` FROM `default`.`src_txn` where `src_txn`.`key` > 200 and `src_txn`.`key` < 250) `partition_mv_1`
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_partitioned_3.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_partitioned_3.q.out
index 65c52be..7543fd4 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_partitioned_3.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_partitioned_3.q.out
@@ -125,7 +125,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       partition columns: key string
       columns: value string
       expanded text: SELECT `value`, `key` FROM (SELECT `src_txn`.`value`, `src_txn`.`key` FROM `default`.`src_txn` where `src_txn`.`key` > 200 and `src_txn`.`key` < 250) `partition_mv_sdp`
diff --git a/ql/src/test/results/clientpositive/llap/selectDistinctStar.q.out b/ql/src/test/results/clientpositive/llap/selectDistinctStar.q.out
index 3fc0074..8f3afe1 100644
--- a/ql/src/test/results/clientpositive/llap/selectDistinctStar.q.out
+++ b/ql/src/test/results/clientpositive/llap/selectDistinctStar.q.out
@@ -1372,15 +1372,15 @@ POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@sdi
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
+  Stage: Stage-0
     Create View
-      columns: key string, value string
       expanded text: select distinct `src`.`key`, `src`.`value` from `default`.`src` order by `src`.`key` limit 2
-      name: default.sdi
       original text: select distinct * from src order by key limit 2
+      columns: key string, value string
+      name: default.sdi
 
 PREHOOK: query: create view sdi as select distinct * from src order by key limit 2
 PREHOOK: type: CREATEVIEW
@@ -3861,15 +3861,15 @@ POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@sdi
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
+  Stage: Stage-0
     Create View
-      columns: key string, value string
       expanded text: select distinct `src`.`key`, `src`.`value` from `default`.`src` order by `src`.`key` limit 2
-      name: default.sdi
       original text: select distinct * from src order by key limit 2
+      columns: key string, value string
+      name: default.sdi
 
 PREHOOK: query: create view sdi as select distinct * from src order by key limit 2
 PREHOOK: type: CREATEVIEW
diff --git a/ql/src/test/results/clientpositive/llap/sketches_materialized_view_safety.q.out b/ql/src/test/results/clientpositive/llap/sketches_materialized_view_safety.q.out
index ff07958..f016486 100644
--- a/ql/src/test/results/clientpositive/llap/sketches_materialized_view_safety.q.out
+++ b/ql/src/test/results/clientpositive/llap/sketches_materialized_view_safety.q.out
@@ -147,7 +147,7 @@ STAGE PLANS:
     Dependency Collection
 
   Stage: Stage-4
-    Create View
+    Create Materialized View
       columns: _c0 string, category char(1), _c2 bigint
       expanded text: select 'no-rewrite-may-happen',`sketch_input`.`category`, count(distinct `sketch_input`.`id`) from `default`.`sketch_input` group by `sketch_input`.`category`
       name: default.mv_1
diff --git a/ql/src/test/results/clientpositive/llap/union_top_level.q.out b/ql/src/test/results/clientpositive/llap/union_top_level.q.out
index f846cb2..5fb0f50 100644
--- a/ql/src/test/results/clientpositive/llap/union_top_level.q.out
+++ b/ql/src/test/results/clientpositive/llap/union_top_level.q.out
@@ -1353,23 +1353,23 @@ POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@union_top_view
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
+  Stage: Stage-0
     Create View
-      columns: key string, value int
       expanded text: select `a`.`key`, `a`.`value` from (select `src`.`key`, 0 as `value` from `default`.`src` where `src`.`key` % 3 == 0 limit 3)`a`
 union all
 select `b`.`key`, `b`.`value` from (select `src`.`key`, 1 as `value` from `default`.`src` where `src`.`key` % 3 == 1 limit 3)`b`
 union all
 select `c`.`key`, `c`.`value` from (select `src`.`key`, 2 as `value` from `default`.`src` where `src`.`key` % 3 == 2 limit 3)`c`
-      name: default.union_top_view
       original text: select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a
 union all
 select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b
 union all
 select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c
+      columns: key string, value int
+      name: default.union_top_view
 
 PREHOOK: query: create view union_top_view as
 select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a
diff --git a/ql/src/test/results/clientpositive/llap/vector_windowing.q.out b/ql/src/test/results/clientpositive/llap/vector_windowing.q.out
index 4636482..34c7e20 100644
--- a/ql/src/test/results/clientpositive/llap/vector_windowing.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_windowing.q.out
@@ -4718,22 +4718,22 @@ PLAN VECTORIZATION:
   enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
 
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
+  Stage: Stage-0
     Create View
-      if not exists: true
-      columns: p_mfgr string, p_brand string, s double
       expanded text: select `part`.`p_mfgr`, `part`.`p_brand`, 
 round(sum(`part`.`p_retailprice`),2) as `s` 
 from `default`.`part` 
 group by `part`.`p_mfgr`, `part`.`p_brand`
-      name: default.mfgr_price_view_n2
+      if not exists: true
       original text: select p_mfgr, p_brand, 
 round(sum(p_retailprice),2) as s 
 from part 
 group by p_mfgr, p_brand
+      columns: p_mfgr string, p_brand string, s double
+      name: default.mfgr_price_view_n2
 
 PREHOOK: query: create view IF NOT EXISTS mfgr_price_view_n2 as 
 select p_mfgr, p_brand, 
@@ -5053,22 +5053,22 @@ PLAN VECTORIZATION:
   enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
 
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
+  Stage: Stage-0
     Create View
-      if not exists: true
-      columns: p_mfgr string, p_brand string, s double
       expanded text: select `part`.`p_mfgr`, `part`.`p_brand`, 
 round(sum(`part`.`p_retailprice`) over w1,2) as `s`
 from `default`.`part` 
 window w1 as (distribute by `part`.`p_mfgr` sort by `part`.`p_name` rows between 2 preceding and current row)
-      name: default.mfgr_brand_price_view_n0
+      if not exists: true
       original text: select p_mfgr, p_brand, 
 round(sum(p_retailprice) over w1,2) as s
 from part 
 window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and current row)
+      columns: p_mfgr string, p_brand string, s double
+      name: default.mfgr_brand_price_view_n0
 
 PREHOOK: query: create view IF NOT EXISTS mfgr_brand_price_view_n0 as 
 select p_mfgr, p_brand, 
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 33546dd..68305a5 100644
--- a/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out
@@ -675,8 +675,8 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v_n5
 Plan optimized by CBO.
 
-Stage-1
-  Create View{"name:":"default.v_n5","original text:":"with cte as (select * from src  order by key limit 5)\nselect * from cte"}
+Stage-0
+  Create View{"original text:":"with cte as (select * from src  order by key limit 5)\nselect * from cte","name:":"default.v_n5"}
 
 PREHOOK: query: 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 f1c245b..b05121d 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
@@ -541,8 +541,8 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v_n1
 Plan optimized by CBO.
 
-Stage-1
-  Create View{"name:":"default.v_n1","original text:":"with cte as (select * from src  order by key limit 5)\nselect * from cte"}
+Stage-0
+  Create View{"original text:":"with cte as (select * from src  order by key limit 5)\nselect * from cte","name:":"default.v_n1"}
 
 PREHOOK: query: explain with cte as (select * from src  order by key limit 5)
 select * from cte