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/02/16 09:29:23 UTC

[hive] branch master updated: HIVE-22747 Break up DDLSemanticAnalyzer - extract Table info and lock analyzers (Miklos Gergely, reviewed by Zoltan Haindrich)

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 d9aa6dc  HIVE-22747 Break up DDLSemanticAnalyzer - extract Table info and lock analyzers (Miklos Gergely, reviewed by Zoltan Haindrich)
d9aa6dc is described below

commit d9aa6dc2b6f4e6610a7e8d32c164d78a2dce9b29
Author: miklosgergely <mg...@cloudera.com>
AuthorDate: Tue Jan 7 12:13:38 2020 +0100

    HIVE-22747 Break up DDLSemanticAnalyzer - extract Table info and lock analyzers (Miklos Gergely, reviewed by Zoltan Haindrich)
---
 .../cli/SemanticAnalysis/HCatSemanticAnalyzer.java |   6 +-
 .../org/apache/hadoop/hive/ql/ddl/DDLUtils.java    |  18 +
 .../ql/ddl/table/info/desc/DescTableAnalyzer.java  | 192 ++++++++
 .../ddl/table/info/{ => desc}/DescTableDesc.java   |   2 +-
 .../table/info/{ => desc}/DescTableOperation.java  |   4 +-
 .../properties/ShowTablePropertiesAnalyzer.java    |  57 +++
 .../properties}/ShowTablePropertiesDesc.java       |   2 +-
 .../properties}/ShowTablePropertiesOperation.java  |   2 +-
 .../info/show/status/ShowTableStatusAnalyzer.java  |  83 ++++
 .../{ => show/status}/ShowTableStatusDesc.java     |  11 +-
 .../status}/ShowTableStatusOperation.java          |   2 +-
 .../table/info/show/tables/ShowTablesAnalyzer.java |  83 ++++
 .../info/{ => show/tables}/ShowTablesDesc.java     |  40 +-
 .../{ => show/tables}/ShowTablesOperation.java     |  48 +-
 .../hive/ql/ddl/table/lock/LockTableAnalyzer.java  |  64 +++
 .../ql/ddl/table/lock/UnlockTableAnalyzer.java     |  60 +++
 .../ddl/table/lock/show/ShowDbLocksAnalyzer.java   |  67 +++
 .../ql/ddl/table/lock/show/ShowLocksAnalyzer.java  |  88 ++++
 .../ddl/table/lock/{ => show}/ShowLocksDesc.java   |   2 +-
 .../table/lock/{ => show}/ShowLocksOperation.java  |   2 +-
 .../show/ShowMaterializedViewsAnalyzer.java        |  78 +++
 .../show/ShowMaterializedViewsDesc.java}           |  35 +-
 .../show/ShowMaterializedViewsOperation.java       |  66 +++
 .../hive/ql/ddl/view/show/ShowViewsAnalyzer.java   |  78 +++
 .../show/ShowViewsDesc.java}                       |  33 +-
 .../hive/ql/ddl/view/show/ShowViewsOperation.java  |  64 +++
 .../hadoop/hive/ql/lockmgr/DbLockManager.java      |   2 +-
 .../org/apache/hadoop/hive/ql/metadata/Hive.java   |  14 +
 .../metadata/formatting/TextMetaDataFormatter.java |   2 +-
 .../hadoop/hive/ql/parse/DDLSemanticAnalyzer.java  | 547 ---------------------
 .../hive/ql/parse/SemanticAnalyzerFactory.java     |   9 -
 .../clientpositive/show_materialized_views.q       |   6 +
 ql/src/test/queries/clientpositive/show_tables.q   |   8 +
 ql/src/test/queries/clientpositive/show_views.q    |   6 +
 .../clientpositive/show_materialized_views.q.out   | 117 +++++
 .../test/results/clientpositive/show_tables.q.out  | 175 +++++++
 .../test/results/clientpositive/show_views.q.out   | 117 +++++
 37 files changed, 1495 insertions(+), 695 deletions(-)

diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java
index f92478c..8277d34 100644
--- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java
+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java
@@ -25,9 +25,9 @@ import org.apache.hadoop.hive.ql.ddl.database.desc.DescDatabaseDesc;
 import org.apache.hadoop.hive.ql.ddl.database.drop.DropDatabaseDesc;
 import org.apache.hadoop.hive.ql.ddl.database.show.ShowDatabasesDesc;
 import org.apache.hadoop.hive.ql.ddl.database.use.SwitchDatabaseDesc;
-import org.apache.hadoop.hive.ql.ddl.table.info.DescTableDesc;
-import org.apache.hadoop.hive.ql.ddl.table.info.ShowTableStatusDesc;
-import org.apache.hadoop.hive.ql.ddl.table.info.ShowTablesDesc;
+import org.apache.hadoop.hive.ql.ddl.table.info.desc.DescTableDesc;
+import org.apache.hadoop.hive.ql.ddl.table.info.show.status.ShowTableStatusDesc;
+import org.apache.hadoop.hive.ql.ddl.table.info.show.tables.ShowTablesDesc;
 import org.apache.hadoop.hive.ql.ddl.table.partition.drop.AlterTableDropPartitionDesc;
 import org.apache.hadoop.hive.ql.ddl.table.partition.show.ShowPartitionsDesc;
 import org.apache.hadoop.hive.ql.ddl.table.storage.AlterTableSetLocationDesc;
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 eb8b858..b82fc5e 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
@@ -41,6 +41,7 @@ import org.apache.hadoop.hive.ql.metadata.Hive;
 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.ReplicationSpec;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -219,4 +220,21 @@ public final class DDLUtils {
 
     throw new SemanticException("Kill query is only supported in HiveServer2 (not hive cli)");
   }
+
+  /**
+   * Get the fully qualified name in the node.
+   * E.g. the node of the form ^(DOT ^(DOT a b) c) will generate a name of the form "a.b.c".
+   */
+  public static String getFQName(ASTNode node) {
+    if (node.getChildCount() == 0) {
+      return node.getText();
+    } else if (node.getChildCount() == 2) {
+      return getFQName((ASTNode) node.getChild(0)) + "." + getFQName((ASTNode) node.getChild(1));
+    } else if (node.getChildCount() == 3) {
+      return getFQName((ASTNode) node.getChild(0)) + "." + getFQName((ASTNode) node.getChild(1)) + "." +
+          getFQName((ASTNode) node.getChild(2));
+    } else {
+      return null;
+    }
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/desc/DescTableAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/desc/DescTableAnalyzer.java
new file mode 100644
index 0000000..cc11d72
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/desc/DescTableAnalyzer.java
@@ -0,0 +1,192 @@
+/*
+ * 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.table.info.desc;
+
+import java.util.Map;
+
+import org.apache.hadoop.hive.common.TableName;
+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.table.partition.PartitionUtils;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory.DDLType;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+/**
+ * Analyzer for table describing commands.
+ *
+ * A query like this will generate a tree as follows
+ *   "describe formatted default.maptable partition (b=100) id;"
+ * TOK_TABTYPE
+ *   TOK_TABNAME --> root for tablename, 2 child nodes mean DB specified
+ *     default
+ *     maptable
+ *   TOK_PARTSPEC  --> root node for partition spec. else columnName
+ *     TOK_PARTVAL
+ *       b
+ *       100
+ *   id           --> root node for columnName
+ * formatted
+ */
+@DDLType(type=HiveParser.TOK_DESCTABLE)
+public class DescTableAnalyzer extends BaseSemanticAnalyzer {
+  public DescTableAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  public void analyzeInternal(ASTNode root) throws SemanticException {
+    ctx.setResFile(ctx.getLocalTmpPath());
+
+    ASTNode tableTypeExpr = (ASTNode) root.getChild(0);
+
+    TableName tableName = getQualifiedTableName((ASTNode)tableTypeExpr.getChild(0));
+    // if database is not the one currently using validate database
+    if (tableName.getDb() != null) {
+      db.validateDatabaseExists(tableName.getDb());
+    }
+    Table table = getTable(tableName);
+
+    // process the second child, if exists, node to get partition spec(s)
+    Map<String, String> partitionSpec = getPartitionSpec(db, tableTypeExpr, tableName);
+    if (partitionSpec != null) {
+      // validate that partition exists
+      PartitionUtils.getPartition(db, table, partitionSpec, true);
+    }
+
+    // process the third child node,if exists, to get partition spec(s)
+    String columnPath = getColumnPath(db, tableTypeExpr, tableName, partitionSpec);
+
+    boolean showColStats = false;
+    boolean isFormatted = false;
+    boolean isExt = false;
+    if (root.getChildCount() == 2) {
+      int descOptions = root.getChild(1).getType();
+      isFormatted = descOptions == HiveParser.KW_FORMATTED;
+      isExt = descOptions == HiveParser.KW_EXTENDED;
+      // in case of "DESCRIBE FORMATTED tablename column_name" statement, colPath will contain tablename.column_name.
+      // If column_name is not specified colPath will be equal to tableName.
+      // This is how we can differentiate if we are describing a table or column.
+      if (columnPath != null && isFormatted) {
+        showColStats = true;
+      }
+    }
+
+    inputs.add(new ReadEntity(table));
+
+    DescTableDesc desc = new DescTableDesc(ctx.getResFile(), tableName, partitionSpec, columnPath, isExt, isFormatted);
+    Task<?> task = TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc));
+    rootTasks.add(task);
+
+    task.setFetchSource(true);
+    String schema = showColStats ? DescTableDesc.COLUMN_STATISTICS_SCHEMA : DescTableDesc.SCHEMA;
+    setFetchTask(createFetchTask(schema));
+  }
+
+  /**
+   * Get the column path.
+   * Return column name if exists, column could be DOT separated.
+   * Example: lintString.$elem$.myint.
+   * Return table name for column name if no column has been specified.
+   */
+  private String getColumnPath(Hive db, ASTNode node, TableName tableName, Map<String, String> partitionSpec)
+      throws SemanticException {
+
+    // if this ast has only one child, then no column name specified.
+    if (node.getChildCount() == 1) {
+      return null;
+    }
+
+    // Second child node could be partitionSpec or column
+    if (node.getChildCount() > 1) {
+      ASTNode columnNode = (partitionSpec == null) ? (ASTNode) node.getChild(1) : (ASTNode) node.getChild(2);
+      if (columnNode != null) {
+        return String.join(".", tableName.getNotEmptyDbTable(), DDLUtils.getFQName(columnNode));
+      }
+    }
+
+    return null;
+  }
+
+  private Map<String, String> getPartitionSpec(Hive db, ASTNode node, TableName tableName) throws SemanticException {
+    // if this node has only one child, then no partition spec specified.
+    if (node.getChildCount() == 1) {
+      return null;
+    }
+
+    // if ast has two children the 2nd child could be partition spec or columnName
+    // if the ast has 3 children, the second *has to* be partition spec
+    if (node.getChildCount() > 2 && (((ASTNode) node.getChild(1)).getType() != HiveParser.TOK_PARTSPEC)) {
+      throw new SemanticException(((ASTNode) node.getChild(1)).getType() + " is not a partition specification");
+    }
+
+    if (((ASTNode) node.getChild(1)).getType() == HiveParser.TOK_PARTSPEC) {
+      ASTNode partNode = (ASTNode) node.getChild(1);
+
+      Table tab = null;
+      try {
+        tab = db.getTable(tableName.getNotEmptyDbTable());
+      } catch (InvalidTableException e) {
+        throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tableName.getNotEmptyDbTable()), e);
+      } catch (HiveException e) {
+        throw new SemanticException(e.getMessage(), e);
+      }
+
+      Map<String, String> partitionSpec = null;
+      try {
+        partitionSpec = getValidatedPartSpec(tab, partNode, db.getConf(), false);
+      } catch (SemanticException e) {
+        // get exception in resolving partition it could be DESCRIBE table key
+        // return null, continue processing for DESCRIBE table key
+        return null;
+      }
+
+      if (partitionSpec != null) {
+        Partition part = null;
+        try {
+          part = db.getPartition(tab, partitionSpec, false);
+        } catch (HiveException e) {
+          // if get exception in finding partition it could be DESCRIBE table key
+          // return null, continue processing for DESCRIBE table key
+          return null;
+        }
+
+        if (part == null) {
+          throw new SemanticException(ErrorMsg.INVALID_PARTITION.getMsg(partitionSpec.toString()));
+        }
+
+        return partitionSpec;
+      }
+    }
+
+    return null;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/DescTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/desc/DescTableDesc.java
similarity index 98%
rename from ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/DescTableDesc.java
rename to ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/desc/DescTableDesc.java
index 0fb14e6..f96a10c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/DescTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/desc/DescTableDesc.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.info;
+package org.apache.hadoop.hive.ql.ddl.table.info.desc;
 
 import java.io.Serializable;
 import java.util.List;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/DescTableOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/desc/DescTableOperation.java
similarity index 99%
rename from ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/DescTableOperation.java
rename to ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/desc/DescTableOperation.java
index 7e467dd..91a9c69 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/DescTableOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/desc/DescTableOperation.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.info;
+package org.apache.hadoop.hive.ql.ddl.table.info.desc;
 
 import java.io.DataOutputStream;
 import java.sql.SQLException;
@@ -66,7 +66,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import com.google.common.collect.Lists;
 
 /**
- * Operation process of dropping a table.
+ * Operation process of describing a table.
  */
 public class DescTableOperation extends DDLOperation<DescTableDesc> {
   public DescTableOperation(DDLOperationContext context, DescTableDesc desc) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/properties/ShowTablePropertiesAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/properties/ShowTablePropertiesAnalyzer.java
new file mode 100644
index 0000000..c399364
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/properties/ShowTablePropertiesAnalyzer.java
@@ -0,0 +1,57 @@
+/*
+ * 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.table.info.show.properties;
+
+import org.apache.hadoop.hive.common.TableName;
+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.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+/**
+ * Analyzer for show table properties commands.
+ */
+@DDLType(type=HiveParser.TOK_SHOW_TBLPROPERTIES)
+public class ShowTablePropertiesAnalyzer extends BaseSemanticAnalyzer {
+  public ShowTablePropertiesAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  public void analyzeInternal(ASTNode root) throws SemanticException {
+    ctx.setResFile(ctx.getLocalTmpPath());
+
+    TableName tableName = getQualifiedTableName((ASTNode) root.getChild(0));
+    String propertyName = (root.getChildCount() > 1) ? unescapeSQLString(root.getChild(1).getText()) : null;
+
+    getTable(tableName); // validate that table exists
+
+    ShowTablePropertiesDesc desc = new ShowTablePropertiesDesc(ctx.getResFile().toString(), tableName, propertyName);
+    Task<DDLWork> task = TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc));
+    rootTasks.add(task);
+
+    task.setFetchSource(true);
+    setFetchTask(createFetchTask(ShowTablePropertiesDesc.SCHEMA));
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTablePropertiesDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/properties/ShowTablePropertiesDesc.java
similarity index 97%
rename from ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTablePropertiesDesc.java
rename to ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/properties/ShowTablePropertiesDesc.java
index 9d3d0b9..062dc2e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTablePropertiesDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/properties/ShowTablePropertiesDesc.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.info;
+package org.apache.hadoop.hive.ql.ddl.table.info.show.properties;
 
 import java.io.Serializable;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTablePropertiesOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/properties/ShowTablePropertiesOperation.java
similarity index 97%
rename from ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTablePropertiesOperation.java
rename to ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/properties/ShowTablePropertiesOperation.java
index 3826bce..d7ab616 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTablePropertiesOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/properties/ShowTablePropertiesOperation.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.info;
+package org.apache.hadoop.hive.ql.ddl.table.info.show.properties;
 
 import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/status/ShowTableStatusAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/status/ShowTableStatusAnalyzer.java
new file mode 100644
index 0000000..77a799c
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/status/ShowTableStatusAnalyzer.java
@@ -0,0 +1,83 @@
+/*
+ * 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.table.info.show.status;
+
+import java.util.Map;
+
+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.table.partition.PartitionUtils;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.HiveTableName;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+/**
+ * Analyzer for show table status commands.
+ */
+@DDLType(type=HiveParser.TOK_SHOW_TABLESTATUS)
+public class ShowTableStatusAnalyzer extends BaseSemanticAnalyzer {
+  public ShowTableStatusAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  public void analyzeInternal(ASTNode root) throws SemanticException {
+    if (root.getChildCount() > 3 || root.getChildCount() < 1) {
+      throw new SemanticException(ErrorMsg.INVALID_AST_TREE.getMsg());
+    }
+
+    ctx.setResFile(ctx.getLocalTmpPath());
+
+    String tableNames = getUnescapedName((ASTNode) root.getChild(0));
+    String dbName = SessionState.get().getCurrentDatabase();
+    Map<String, String> partitionSpec = null;
+    if (root.getChildCount() > 1) {
+      for (int i = 1; i < root.getChildCount(); i++) {
+        ASTNode child = (ASTNode) root.getChild(i);
+        if (child.getToken().getType() == HiveParser.Identifier) {
+          dbName = unescapeIdentifier(child.getText());
+        } else if (child.getToken().getType() == HiveParser.TOK_PARTSPEC) {
+          partitionSpec = getValidatedPartSpec(getTable(tableNames), child, conf, false);
+        } else {
+          throw new SemanticException(ErrorMsg.INVALID_AST_TREE.getMsg(
+              child.toStringTree() + " , Invalid token " + child.getToken().getType()));
+        }
+      }
+    }
+
+    if (partitionSpec != null) {
+      // validate that partition exists
+      PartitionUtils.getPartition(db, getTable(HiveTableName.of(tableNames)), partitionSpec, true);
+    }
+
+    ShowTableStatusDesc desc = new ShowTableStatusDesc(ctx.getResFile(), dbName, tableNames, partitionSpec);
+    Task<DDLWork> task = TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc));
+    rootTasks.add(task);
+
+    task.setFetchSource(true);
+    setFetchTask(createFetchTask(ShowTableStatusDesc.SCHEMA));
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTableStatusDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/status/ShowTableStatusDesc.java
similarity index 87%
copy from ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTableStatusDesc.java
copy to ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/status/ShowTableStatusDesc.java
index b53d138..6726518 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTableStatusDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/status/ShowTableStatusDesc.java
@@ -16,11 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.info;
+package org.apache.hadoop.hive.ql.ddl.table.info.show.status;
 
 import java.io.Serializable;
 import java.util.Map;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.ddl.DDLDesc;
 import org.apache.hadoop.hive.ql.plan.Explain;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
@@ -39,12 +40,8 @@ public class ShowTableStatusDesc implements DDLDesc, Serializable {
   private final String pattern;
   private final Map<String, String> partSpec;
 
-  public ShowTableStatusDesc(String resFile, String dbName, String pattern) {
-    this(resFile, dbName, pattern, null);
-  }
-
-  public ShowTableStatusDesc(String resFile, String dbName, String pattern, Map<String, String> partSpec) {
-    this.resFile = resFile;
+  public ShowTableStatusDesc(Path resFile, String dbName, String pattern, Map<String, String> partSpec) {
+    this.resFile = resFile.toString();
     this.dbName = dbName;
     this.pattern = pattern;
     this.partSpec = partSpec;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTableStatusOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/status/ShowTableStatusOperation.java
similarity index 98%
rename from ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTableStatusOperation.java
rename to ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/status/ShowTableStatusOperation.java
index bc8ec66..914e63d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTableStatusOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/status/ShowTableStatusOperation.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.info;
+package org.apache.hadoop.hive.ql.ddl.table.info.show.status;
 
 import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/tables/ShowTablesAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/tables/ShowTablesAnalyzer.java
new file mode 100644
index 0000000..dd2b6e6
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/tables/ShowTablesAnalyzer.java
@@ -0,0 +1,83 @@
+/*
+ * 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.table.info.show.tables;
+
+import org.apache.hadoop.hive.metastore.TableType;
+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.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+/**
+ * Analyzer for show tables commands.
+ */
+@DDLType(type=HiveParser.TOK_SHOWTABLES)
+public class ShowTablesAnalyzer extends BaseSemanticAnalyzer {
+  public ShowTablesAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  public void analyzeInternal(ASTNode root) throws SemanticException {
+    if (root.getChildCount() > 4) {
+      throw new SemanticException(ErrorMsg.INVALID_AST_TREE.getMsg(root.toStringTree()));
+    }
+
+    ctx.setResFile(ctx.getLocalTmpPath());
+
+    String dbName = SessionState.get().getCurrentDatabase();
+    String tableNames = null;
+    TableType tableTypeFilter = null;
+    boolean isExtended = false;
+    for (int i = 0; i < root.getChildCount(); i++) {
+      ASTNode child = (ASTNode) root.getChild(i);
+      if (child.getType() == HiveParser.TOK_FROM) { // Specifies a DB
+        dbName = unescapeIdentifier(root.getChild(++i).getText());
+        db.validateDatabaseExists(dbName);
+      } else if (child.getType() == HiveParser.TOK_TABLE_TYPE) { // Filter on table type
+        String tableType = unescapeIdentifier(child.getChild(0).getText());
+        if (!"table_type".equalsIgnoreCase(tableType)) {
+          throw new SemanticException("SHOW TABLES statement only allows equality filter on table_type value");
+        }
+        tableTypeFilter = TableType.valueOf(unescapeSQLString(child.getChild(1).getText()));
+      } else if (child.getType() == HiveParser.KW_EXTENDED) { // Include table type
+        isExtended = true;
+      } else { // Uses a pattern
+        tableNames = unescapeSQLString(child.getText());
+      }
+    }
+
+    inputs.add(new ReadEntity(getDatabase(dbName)));
+
+    ShowTablesDesc desc = new ShowTablesDesc(ctx.getResFile(), dbName, tableNames, tableTypeFilter, isExtended);
+    Task<DDLWork> task = TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc));
+    rootTasks.add(task);
+
+    task.setFetchSource(true);
+    setFetchTask(createFetchTask(desc.getSchema()));
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTablesDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/tables/ShowTablesDesc.java
similarity index 72%
rename from ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTablesDesc.java
rename to ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/tables/ShowTablesDesc.java
index 44c1f09..99c1118 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTablesDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/tables/ShowTablesDesc.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.info;
+package org.apache.hadoop.hive.ql.ddl.table.info.show.tables;
 
 import java.io.Serializable;
 
@@ -35,42 +35,17 @@ public class ShowTablesDesc implements DDLDesc, Serializable {
 
   private static final String TABLES_VIEWS_SCHEMA = "tab_name#string";
   private static final String EXTENDED_TABLES_SCHEMA = "tab_name,table_type#string,string";
-  private static final String MATERIALIZED_VIEWS_SCHEMA = "mv_name,rewrite_enabled,mode#string:string:string";
 
   private final String resFile;
   private final String dbName;
   private final String pattern;
-  private final TableType type;
   private final TableType typeFilter;
   private final boolean isExtended;
 
-  public ShowTablesDesc(Path resFile) {
-    this(resFile, null, null, null, null, false);
-  }
-
-  public ShowTablesDesc(Path resFile, String dbName) {
-    this(resFile, dbName, null, null, null, false);
-  }
-
-  public ShowTablesDesc(Path resFile, String dbName, TableType type) {
-    this(resFile, dbName, null, type, null, false);
-  }
-
   public ShowTablesDesc(Path resFile, String dbName, String pattern, TableType typeFilter, boolean isExtended) {
-    this(resFile, dbName, pattern, null, typeFilter, isExtended);
-  }
-
-  public ShowTablesDesc(Path resFile, String dbName, String pattern, TableType type) {
-    this(resFile, dbName, pattern, type, null, false);
-  }
-
-
-  public ShowTablesDesc(Path resFile, String dbName, String pattern, TableType type, TableType typeFilter,
-      boolean isExtended) {
     this.resFile = resFile.toString();
     this.dbName = dbName;
     this.pattern = pattern;
-    this.type = type;
     this.typeFilter = typeFilter;
     this.isExtended = isExtended;
   }
@@ -80,11 +55,6 @@ public class ShowTablesDesc implements DDLDesc, Serializable {
     return pattern;
   }
 
-  @Explain(displayName = "type")
-  public TableType getType() {
-    return type;
-  }
-
   @Explain(displayName = "result file", explainLevels = { Level.EXTENDED })
   public String getResFile() {
     return resFile;
@@ -101,15 +71,17 @@ public class ShowTablesDesc implements DDLDesc, Serializable {
     return isExtended;
   }
 
+  /** For explain only. */
   @Explain(displayName = "table type filter", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  public String getTypeFilterString() {
+    return typeFilter.name();
+  }
+
   public TableType getTypeFilter() {
     return typeFilter;
   }
 
   public String getSchema() {
-    if (type != null && type == TableType.MATERIALIZED_VIEW) {
-      return MATERIALIZED_VIEWS_SCHEMA;
-    }
     return isExtended ? EXTENDED_TABLES_SCHEMA : TABLES_VIEWS_SCHEMA;
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTablesOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/tables/ShowTablesOperation.java
similarity index 58%
rename from ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTablesOperation.java
rename to ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/tables/ShowTablesOperation.java
index c174e09..4846d29 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTablesOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/show/tables/ShowTablesOperation.java
@@ -16,9 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.info;
+package org.apache.hadoop.hive.ql.ddl.table.info.show.tables;
 
 import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
 
 import java.io.DataOutputStream;
 import java.util.ArrayList;
@@ -28,14 +29,12 @@ import java.util.List;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 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.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
-import org.apache.hadoop.io.IOUtils;
 
 /**
  * Operation process showing the tables.
@@ -47,7 +46,6 @@ public class ShowTablesOperation extends DDLOperation<ShowTablesDesc> {
 
   @Override
   public int execute() throws HiveException {
-    TableType type       = desc.getType(); // null for tables, VIRTUAL_VIEW for views, MATERIALIZED_VIEW for MVs
     String dbName        = desc.getDbName();
     String pattern       = desc.getPattern(); // if null, all tables/views are returned
     TableType typeFilter = desc.getTypeFilter();
@@ -58,55 +56,33 @@ public class ShowTablesOperation extends DDLOperation<ShowTablesDesc> {
       throw new HiveException(ErrorMsg.DATABASE_NOT_EXISTS, dbName);
     }
 
-    LOG.debug("pattern: {}", pattern);
-    LOG.debug("typeFilter: {}", typeFilter);
-
     List<String> tableNames  = null;
     List<Table> tableObjects = null;
-    if (type == null) {
-      if (isExtended) {
-        tableObjects = new ArrayList<>();
-        tableObjects.addAll(context.getDb().getTableObjectsByType(dbName, pattern, typeFilter));
-        LOG.debug("Found {} table(s) matching the SHOW EXTENDED TABLES statement.", tableObjects.size());
-      } else {
-        tableNames = context.getDb().getTablesByType(dbName, pattern, typeFilter);
-        LOG.debug("Found {} table(s) matching the SHOW TABLES statement.", tableNames.size());
-      }
-    } else if (type == TableType.MATERIALIZED_VIEW) {
+    if (isExtended) {
       tableObjects = new ArrayList<>();
-      tableObjects.addAll(context.getDb().getMaterializedViewObjectsByPattern(dbName, pattern));
-      LOG.debug("Found {} materialized view(s) matching the SHOW MATERIALIZED VIEWS statement.", tableObjects.size());
-    } else if (type == TableType.VIRTUAL_VIEW) {
-      tableNames = context.getDb().getTablesByType(dbName, pattern, type);
-      LOG.debug("Found {} view(s) matching the SHOW VIEWS statement.", tableNames.size());
+      tableObjects.addAll(context.getDb().getTableObjectsByType(dbName, pattern, typeFilter));
+      LOG.debug("Found {} table(s) matching the SHOW EXTENDED TABLES statement.", tableObjects.size());
     } else {
-      throw new HiveException("Option not recognized in SHOW TABLES/VIEWS/MATERIALIZED VIEWS");
+      tableNames = context.getDb().getTablesByType(dbName, pattern, typeFilter);
+      LOG.debug("Found {} table(s) matching the SHOW TABLES statement.", tableNames.size());
     }
 
-    // write the results in the file
-    DataOutputStream outStream = null;
-    try {
-      Path resFile = new Path(resultsFile);
-      FileSystem fs = resFile.getFileSystem(context.getConf());
-      outStream = fs.create(resFile);
-      // Sort by name and print
+    try (DataOutputStream os = DDLUtils.getOutputStream(new Path(resultsFile), context)) {
       if (tableNames != null) {
         SortedSet<String> sortedSet = new TreeSet<String>(tableNames);
-        context.getFormatter().showTables(outStream, sortedSet);
+        context.getFormatter().showTables(os, sortedSet);
       } else {
         Collections.sort(tableObjects, Comparator.comparing(Table::getTableName));
         if (isExtended) {
-          context.getFormatter().showTablesExtended(outStream, tableObjects);
+          context.getFormatter().showTablesExtended(os, tableObjects);
         } else {
-          context.getFormatter().showMaterializedViews(outStream, tableObjects);
+          context.getFormatter().showMaterializedViews(os, tableObjects);
         }
       }
-      outStream.close();
     } catch (Exception e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR, "in database" + dbName);
-    } finally {
-      IOUtils.closeStream(outStream);
     }
+
     return 0;
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/LockTableAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/LockTableAnalyzer.java
new file mode 100644
index 0000000..8282966
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/LockTableAnalyzer.java
@@ -0,0 +1,64 @@
+/*
+ * 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.table.lock;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory.DDLType;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+/**
+ * Analyzer for lock table commands.
+ */
+@DDLType(type=HiveParser.TOK_LOCKTABLE)
+public class LockTableAnalyzer extends BaseSemanticAnalyzer {
+  public LockTableAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  public void analyzeInternal(ASTNode root) throws SemanticException {
+    String tableName = getUnescapedName((ASTNode) root.getChild(0)).toLowerCase();
+    String mode = unescapeIdentifier(root.getChild(1).getText().toUpperCase());
+    List<Map<String, String>> partitionSpecs = getPartitionSpecs(getTable(tableName), root);
+
+    // We only can have a single partition spec
+    assert (partitionSpecs.size() <= 1);
+    Map<String, String> partitionSpec = null;
+    if (partitionSpecs.size() > 0) {
+      partitionSpec = partitionSpecs.get(0);
+    }
+
+    LockTableDesc desc = new LockTableDesc(tableName, mode, partitionSpec,
+        HiveConf.getVar(conf, ConfVars.HIVEQUERYID), ctx.getCmd());
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
+
+    // Need to initialize the lock manager
+    ctx.setNeedLockMgr(true);
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/UnlockTableAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/UnlockTableAnalyzer.java
new file mode 100644
index 0000000..18a838b
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/UnlockTableAnalyzer.java
@@ -0,0 +1,60 @@
+/*
+ * 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.table.lock;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory.DDLType;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+/**
+ * Analyzer for unlock table commands.
+ */
+@DDLType(type=HiveParser.TOK_UNLOCKTABLE)
+public class UnlockTableAnalyzer extends BaseSemanticAnalyzer {
+  public UnlockTableAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  public void analyzeInternal(ASTNode root) throws SemanticException {
+    String tableName = getUnescapedName((ASTNode) root.getChild(0));
+    List<Map<String, String>> partitionSpecs = getPartitionSpecs(getTable(tableName), root);
+
+    // We only can have a single partition spec
+    assert (partitionSpecs.size() <= 1);
+    Map<String, String> partitionSpec = null;
+    if (partitionSpecs.size() > 0) {
+      partitionSpec = partitionSpecs.get(0);
+    }
+
+    UnlockTableDesc desc = new UnlockTableDesc(tableName, partitionSpec);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
+
+    // Need to initialize the lock manager
+    ctx.setNeedLockMgr(true);
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/show/ShowDbLocksAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/show/ShowDbLocksAnalyzer.java
new file mode 100644
index 0000000..c6b7f42
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/show/ShowDbLocksAnalyzer.java
@@ -0,0 +1,67 @@
+/*
+ * 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.table.lock.show;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory.DDLType;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
+import org.apache.hadoop.hive.ql.lockmgr.LockException;
+import org.apache.hadoop.hive.ql.lockmgr.TxnManagerFactory;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+/**
+ * Analyzer for show DB locks commands.
+ */
+@DDLType(type=HiveParser.TOK_SHOWDBLOCKS)
+public class ShowDbLocksAnalyzer extends BaseSemanticAnalyzer {
+  public ShowDbLocksAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  public void analyzeInternal(ASTNode root) throws SemanticException {
+    ctx.setResFile(ctx.getLocalTmpPath());
+
+    String dbName = stripQuotes(root.getChild(0).getText());
+    boolean isExtended = (root.getChildCount() > 1);
+
+    HiveTxnManager txnManager = null;
+    try {
+      txnManager = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    } catch (LockException e) {
+      throw new SemanticException(e.getMessage());
+    }
+
+    ShowLocksDesc desc = new ShowLocksDesc(ctx.getResFile(), dbName, isExtended, txnManager.useNewShowLocksFormat());
+    Task<DDLWork> task = TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc));
+    rootTasks.add(task);
+
+    task.setFetchSource(true);
+    setFetchTask(createFetchTask(desc.getSchema()));
+
+    // Need to initialize the lock manager
+    ctx.setNeedLockMgr(true);
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/show/ShowLocksAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/show/ShowLocksAnalyzer.java
new file mode 100644
index 0000000..709ce47
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/show/ShowLocksAnalyzer.java
@@ -0,0 +1,88 @@
+/*
+ * 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.table.lock.show;
+
+import java.util.Map;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory.DDLType;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
+import org.apache.hadoop.hive.ql.lockmgr.LockException;
+import org.apache.hadoop.hive.ql.lockmgr.TxnManagerFactory;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+/**
+ * Analyzer for show locks commands.
+ */
+@DDLType(type=HiveParser.TOK_SHOWLOCKS)
+public class ShowLocksAnalyzer extends BaseSemanticAnalyzer {
+  public ShowLocksAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  public void analyzeInternal(ASTNode root) throws SemanticException {
+    ctx.setResFile(ctx.getLocalTmpPath());
+
+    String tableName = null;
+    Map<String, String> partitionSpec = null;
+    boolean isExtended = false;
+    if (root.getChildCount() >= 1) {
+      // table for which show locks is being executed
+      for (int i = 0; i < root.getChildCount(); i++) {
+        ASTNode child = (ASTNode) root.getChild(i);
+        if (child.getType() == HiveParser.TOK_TABTYPE) {
+          tableName = DDLUtils.getFQName((ASTNode) child.getChild(0));
+          // get partition metadata if partition specified
+          if (child.getChildCount() == 2) {
+            ASTNode partitionSpecNode = (ASTNode) child.getChild(1);
+            partitionSpec = getValidatedPartSpec(getTable(tableName), partitionSpecNode, conf, false);
+          }
+        } else if (child.getType() == HiveParser.KW_EXTENDED) {
+          isExtended = true;
+        }
+      }
+    }
+
+    HiveTxnManager txnManager = null;
+    try {
+      txnManager = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    } catch (LockException e) {
+      throw new SemanticException(e.getMessage());
+    }
+
+    ShowLocksDesc desc = new ShowLocksDesc(ctx.getResFile(), tableName, partitionSpec, isExtended,
+        txnManager.useNewShowLocksFormat());
+    Task<DDLWork> task = TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc));
+    rootTasks.add(task);
+
+    task.setFetchSource(true);
+    setFetchTask(createFetchTask(desc.getSchema()));
+
+    // Need to initialize the lock manager
+    ctx.setNeedLockMgr(true);
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/show/ShowLocksDesc.java
similarity index 98%
rename from ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksDesc.java
rename to ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/show/ShowLocksDesc.java
index 8ede350..898d3b8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/show/ShowLocksDesc.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.lock;
+package org.apache.hadoop.hive.ql.ddl.table.lock.show;
 
 import java.io.Serializable;
 import java.util.Map;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/show/ShowLocksOperation.java
similarity index 99%
rename from ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java
rename to ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/show/ShowLocksOperation.java
index 506b7cf..0f11aad 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/show/ShowLocksOperation.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.lock;
+package org.apache.hadoop.hive.ql.ddl.table.lock.show;
 
 import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/show/ShowMaterializedViewsAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/show/ShowMaterializedViewsAnalyzer.java
new file mode 100644
index 0000000..f2779da
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/show/ShowMaterializedViewsAnalyzer.java
@@ -0,0 +1,78 @@
+/*
+ * 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.materialized.show;
+
+import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory.DDLType;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+/**
+ * Analyzer for show materialized views commands.
+ */
+@DDLType(type=HiveParser.TOK_SHOWMATERIALIZEDVIEWS)
+public class ShowMaterializedViewsAnalyzer extends BaseSemanticAnalyzer {
+  public ShowMaterializedViewsAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  public void analyzeInternal(ASTNode root) throws SemanticException {
+    if (root.getChildCount() > 3) {
+      throw new SemanticException(ErrorMsg.GENERIC_ERROR.getMsg());
+    }
+
+    ctx.setResFile(ctx.getLocalTmpPath());
+
+    String dbName = SessionState.get().getCurrentDatabase();
+    String viewNames = null;
+    switch (root.getChildCount()) {
+    case 1: // Uses a pattern
+      viewNames = unescapeSQLString(root.getChild(0).getText());
+      break;
+    case 2: // Specifies a DB
+      assert (root.getChild(0).getType() == HiveParser.TOK_FROM);
+      dbName = unescapeIdentifier(root.getChild(1).getText());
+      db.validateDatabaseExists(dbName);
+      break;
+    case 3: // Uses a pattern and specifies a DB
+      assert (root.getChild(0).getType() == HiveParser.TOK_FROM);
+      dbName = unescapeIdentifier(root.getChild(1).getText());
+      viewNames = unescapeSQLString(root.getChild(2).getText());
+      db.validateDatabaseExists(dbName);
+      break;
+    default: // No pattern or DB
+      break;
+    }
+
+    ShowMaterializedViewsDesc desc = new ShowMaterializedViewsDesc(ctx.getResFile(), dbName, viewNames);
+    Task<DDLWork> task = TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc));
+    rootTasks.add(task);
+
+    task.setFetchSource(true);
+    setFetchTask(createFetchTask(ShowMaterializedViewsDesc.SCHEMA));
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTableStatusDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/show/ShowMaterializedViewsDesc.java
similarity index 58%
copy from ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTableStatusDesc.java
copy to ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/show/ShowMaterializedViewsDesc.java
index b53d138..0bb856a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTableStatusDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/show/ShowMaterializedViewsDesc.java
@@ -16,38 +16,32 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.info;
+package org.apache.hadoop.hive.ql.ddl.view.materialized.show;
 
 import java.io.Serializable;
-import java.util.Map;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.ddl.DDLDesc;
 import org.apache.hadoop.hive.ql.plan.Explain;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
 
 /**
- * DDL task description for SHOW TABLE STATUS commands.
+ * DDL task description for SHOW MATERIALIZED VIEWS commands.
  */
-@Explain(displayName = "Show Table Status", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
-public class ShowTableStatusDesc implements DDLDesc, Serializable {
+@Explain(displayName = "Show Materialized Views", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+public class ShowMaterializedViewsDesc implements DDLDesc, Serializable {
   private static final long serialVersionUID = 1L;
 
-  public static final String SCHEMA = "tab_name#string";
+  public static final String SCHEMA = "mv_name,rewrite_enabled,mode#string:string:string";
 
   private final String resFile;
   private final String dbName;
   private final String pattern;
-  private final Map<String, String> partSpec;
 
-  public ShowTableStatusDesc(String resFile, String dbName, String pattern) {
-    this(resFile, dbName, pattern, null);
-  }
-
-  public ShowTableStatusDesc(String resFile, String dbName, String pattern, Map<String, String> partSpec) {
-    this.resFile = resFile;
+  public ShowMaterializedViewsDesc(Path resFile, String dbName, String pattern) {
+    this.resFile = resFile.toString();
     this.dbName = dbName;
     this.pattern = pattern;
-    this.partSpec = partSpec;
   }
 
   @Explain(displayName = "pattern")
@@ -55,22 +49,13 @@ public class ShowTableStatusDesc implements DDLDesc, Serializable {
     return pattern;
   }
 
+  @Explain(displayName = "result file", explainLevels = { Level.EXTENDED })
   public String getResFile() {
     return resFile;
   }
 
-  @Explain(displayName = "result file", explainLevels = { Level.EXTENDED })
-  public String getResFileString() {
-    return getResFile();
-  }
-
-  @Explain(displayName = "database", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  @Explain(displayName = "database name", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
   public String getDbName() {
     return dbName;
   }
-
-  @Explain(displayName = "partition", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
-  public Map<String, String> getPartSpec() {
-    return partSpec;
-  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/show/ShowMaterializedViewsOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/show/ShowMaterializedViewsOperation.java
new file mode 100644
index 0000000..792b352
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/show/ShowMaterializedViewsOperation.java
@@ -0,0 +1,66 @@
+/*
+ * 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.materialized.show;
+
+import java.io.DataOutputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.ErrorMsg;
+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.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Table;
+
+/**
+ * Operation process showing the materialized views.
+ */
+public class ShowMaterializedViewsOperation extends DDLOperation<ShowMaterializedViewsDesc> {
+  public ShowMaterializedViewsOperation(DDLOperationContext context, ShowMaterializedViewsDesc desc) {
+    super(context, desc);
+  }
+
+  @Override
+  public int execute() throws HiveException {
+    String dbName = desc.getDbName();
+    String pattern = desc.getPattern(); // if null, all tables/views are returned
+    String resultsFile = desc.getResFile();
+
+    if (!context.getDb().databaseExists(dbName)) {
+      throw new HiveException(ErrorMsg.DATABASE_NOT_EXISTS, dbName);
+    }
+
+    // the returned list is not sortable as it is immutable, thus it must be put into a new ArrayList
+    List<Table> tableObjects = new ArrayList<>(context.getDb().getMaterializedViewObjectsByPattern(dbName, pattern));
+    LOG.debug("Found {} materialized view(s) matching the SHOW MATERIALIZED VIEWS statement.", tableObjects.size());
+
+    try (DataOutputStream os = DDLUtils.getOutputStream(new Path(resultsFile), context)) {
+      Collections.sort(tableObjects, Comparator.comparing(Table::getTableName));
+      context.getFormatter().showMaterializedViews(os, tableObjects);
+    } catch (Exception e) {
+      throw new HiveException(e, ErrorMsg.GENERIC_ERROR, "in database" + dbName);
+    }
+
+    return 0;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/show/ShowViewsAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/show/ShowViewsAnalyzer.java
new file mode 100644
index 0000000..0467196
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/show/ShowViewsAnalyzer.java
@@ -0,0 +1,78 @@
+/*
+ * 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.show;
+
+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.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+/**
+ * Analyzer for show views commands.
+ */
+@DDLType(type=HiveParser.TOK_SHOWVIEWS)
+public class ShowViewsAnalyzer extends BaseSemanticAnalyzer {
+  public ShowViewsAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  public void analyzeInternal(ASTNode root) throws SemanticException {
+    if (root.getChildCount() > 3) {
+      throw new SemanticException(ErrorMsg.GENERIC_ERROR.getMsg());
+    }
+
+    ctx.setResFile(ctx.getLocalTmpPath());
+
+    String dbName = SessionState.get().getCurrentDatabase();
+    String viewNames = null;
+    switch (root.getChildCount()) {
+    case 1: // Uses a pattern
+      viewNames = unescapeSQLString(root.getChild(0).getText());
+      break;
+    case 2: // Specifies a DB
+      assert (root.getChild(0).getType() == HiveParser.TOK_FROM);
+      dbName = unescapeIdentifier(root.getChild(1).getText());
+      db.validateDatabaseExists(dbName);
+      break;
+    case 3: // Uses a pattern and specifies a DB
+      assert (root.getChild(0).getType() == HiveParser.TOK_FROM);
+      dbName = unescapeIdentifier(root.getChild(1).getText());
+      viewNames = unescapeSQLString(root.getChild(2).getText());
+      db.validateDatabaseExists(dbName);
+      break;
+    default: // No pattern or DB
+      break;
+    }
+
+    ShowViewsDesc desc = new ShowViewsDesc(ctx.getResFile(), dbName, viewNames);
+    Task<DDLWork> task = TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc));
+    rootTasks.add(task);
+
+    task.setFetchSource(true);
+    setFetchTask(createFetchTask(ShowViewsDesc.SCHEMA));
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTableStatusDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/show/ShowViewsDesc.java
similarity index 62%
rename from ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTableStatusDesc.java
rename to ql/src/java/org/apache/hadoop/hive/ql/ddl/view/show/ShowViewsDesc.java
index b53d138..b3d683e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/ShowTableStatusDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/show/ShowViewsDesc.java
@@ -16,20 +16,20 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.info;
+package org.apache.hadoop.hive.ql.ddl.view.show;
 
 import java.io.Serializable;
-import java.util.Map;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.ddl.DDLDesc;
 import org.apache.hadoop.hive.ql.plan.Explain;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
 
 /**
- * DDL task description for SHOW TABLE STATUS commands.
+ * DDL task description for SHOW VIEWS commands.
  */
-@Explain(displayName = "Show Table Status", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
-public class ShowTableStatusDesc implements DDLDesc, Serializable {
+@Explain(displayName = "Show Views", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+public class ShowViewsDesc implements DDLDesc, Serializable {
   private static final long serialVersionUID = 1L;
 
   public static final String SCHEMA = "tab_name#string";
@@ -37,17 +37,11 @@ public class ShowTableStatusDesc implements DDLDesc, Serializable {
   private final String resFile;
   private final String dbName;
   private final String pattern;
-  private final Map<String, String> partSpec;
 
-  public ShowTableStatusDesc(String resFile, String dbName, String pattern) {
-    this(resFile, dbName, pattern, null);
-  }
-
-  public ShowTableStatusDesc(String resFile, String dbName, String pattern, Map<String, String> partSpec) {
-    this.resFile = resFile;
+  public ShowViewsDesc(Path resFile, String dbName, String pattern) {
+    this.resFile = resFile.toString();
     this.dbName = dbName;
     this.pattern = pattern;
-    this.partSpec = partSpec;
   }
 
   @Explain(displayName = "pattern")
@@ -55,22 +49,13 @@ public class ShowTableStatusDesc implements DDLDesc, Serializable {
     return pattern;
   }
 
+  @Explain(displayName = "result file", explainLevels = { Level.EXTENDED })
   public String getResFile() {
     return resFile;
   }
 
-  @Explain(displayName = "result file", explainLevels = { Level.EXTENDED })
-  public String getResFileString() {
-    return getResFile();
-  }
-
-  @Explain(displayName = "database", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  @Explain(displayName = "database name", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
   public String getDbName() {
     return dbName;
   }
-
-  @Explain(displayName = "partition", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
-  public Map<String, String> getPartSpec() {
-    return partSpec;
-  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/show/ShowViewsOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/show/ShowViewsOperation.java
new file mode 100644
index 0000000..7962551
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/show/ShowViewsOperation.java
@@ -0,0 +1,64 @@
+/*
+ * 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.show;
+
+import java.io.DataOutputStream;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.hadoop.fs.Path;
+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;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+/**
+ * Operation process showing the views.
+ */
+public class ShowViewsOperation extends DDLOperation<ShowViewsDesc> {
+  public ShowViewsOperation(DDLOperationContext context, ShowViewsDesc desc) {
+    super(context, desc);
+  }
+
+  @Override
+  public int execute() throws HiveException {
+    String dbName = desc.getDbName();
+    String pattern = desc.getPattern(); // if null, all tables/views are returned
+    String resultsFile = desc.getResFile();
+
+    if (!context.getDb().databaseExists(dbName)) {
+      throw new HiveException(ErrorMsg.DATABASE_NOT_EXISTS, dbName);
+    }
+
+    List<String> tableNames = context.getDb().getTablesByType(dbName, pattern, TableType.VIRTUAL_VIEW);
+    LOG.debug("Found {} view(s) matching the SHOW VIEWS statement.", tableNames.size());
+
+    try (DataOutputStream os = DDLUtils.getOutputStream(new Path(resultsFile), context)) {
+      SortedSet<String> sortedSet = new TreeSet<String>(tableNames);
+      context.getFormatter().showTables(os, sortedSet);
+    } catch (Exception e) {
+      throw new HiveException(e, ErrorMsg.GENERIC_ERROR, "in database" + dbName);
+    }
+
+    return 0;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java
index a8b9653..4b6bc3e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java
@@ -26,8 +26,8 @@ import org.apache.hadoop.hive.common.metrics.common.Metrics;
 import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
 import org.apache.hadoop.hive.metastore.api.*;
-import org.apache.hadoop.hive.ql.ddl.table.lock.ShowLocksOperation;
 import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.ddl.table.lock.show.ShowLocksOperation;
 import org.apache.hadoop.hive.ql.DriverState;
 import org.apache.thrift.TException;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index 945eafc..1eb9c12 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -192,6 +192,7 @@ import org.apache.hadoop.hive.ql.log.PerfLogger;
 import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveAugmentMaterializationRule;
 import org.apache.hadoop.hive.ql.optimizer.listbucketingpruner.ListBucketingPrunerUtils;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.plan.LoadTableDesc.LoadFileType;
 import org.apache.hadoop.hive.ql.session.CreateTableAutomaticGrant;
@@ -2094,6 +2095,19 @@ public class Hive {
     }
   }
 
+  public void validateDatabaseExists(String databaseName) throws SemanticException {
+    boolean exists;
+    try {
+      exists = databaseExists(databaseName);
+    } catch (HiveException e) {
+      throw new SemanticException(ErrorMsg.DATABASE_NOT_EXISTS.getMsg(databaseName), e);
+    }
+
+    if (!exists) {
+      throw new SemanticException(ErrorMsg.DATABASE_NOT_EXISTS.getMsg(databaseName));
+    }
+  }
+
   /**
    * Query metadata to see if a database with the given name already exists.
    *
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
index 6b35581..d0d40af 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
@@ -46,7 +46,7 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
-import org.apache.hadoop.hive.ql.ddl.table.info.DescTableDesc;
+import org.apache.hadoop.hive.ql.ddl.table.info.desc.DescTableDesc;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.metadata.CheckConstraint;
 import org.apache.hadoop.hive.ql.metadata.DefaultConstraint;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index fd76ada..eee81a3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -55,13 +55,6 @@ import org.apache.hadoop.hive.ql.ddl.DDLWork;
 import org.apache.hadoop.hive.ql.ddl.privilege.PrincipalDesc;
 import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableDesc;
 import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
-import org.apache.hadoop.hive.ql.ddl.table.info.DescTableDesc;
-import org.apache.hadoop.hive.ql.ddl.table.info.ShowTablePropertiesDesc;
-import org.apache.hadoop.hive.ql.ddl.table.info.ShowTableStatusDesc;
-import org.apache.hadoop.hive.ql.ddl.table.info.ShowTablesDesc;
-import org.apache.hadoop.hive.ql.ddl.table.lock.LockTableDesc;
-import org.apache.hadoop.hive.ql.ddl.table.lock.ShowLocksDesc;
-import org.apache.hadoop.hive.ql.ddl.table.lock.UnlockTableDesc;
 import org.apache.hadoop.hive.ql.ddl.table.misc.AlterTableRenameDesc;
 import org.apache.hadoop.hive.ql.ddl.table.misc.AlterTableSetOwnerDesc;
 import org.apache.hadoop.hive.ql.ddl.table.misc.AlterTableSetPropertiesDesc;
@@ -96,20 +89,13 @@ import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
-import org.apache.hadoop.hive.ql.lockmgr.LockException;
-import org.apache.hadoop.hive.ql.lockmgr.TxnManagerFactory;
 import org.apache.hadoop.hive.ql.metadata.DefaultConstraint;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
 import org.apache.hadoop.hive.ql.metadata.NotNullConstraint;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.authorization.AuthorizationParseUtils;
-import org.apache.hadoop.hive.ql.parse.type.ExprNodeTypeCheck;
-import org.apache.hadoop.hive.ql.parse.type.TypeCheckCtx;
-import org.apache.hadoop.hive.ql.parse.type.TypeCheckProcFactory;
 import org.apache.hadoop.hive.ql.plan.BasicStatsWork;
 import org.apache.hadoop.hive.ql.plan.ColumnStatsUpdateWork;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
@@ -128,15 +114,12 @@ import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
 import org.apache.hadoop.mapred.InputFormat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * DDLSemanticAnalyzer.
  *
  */
 public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
-  private static final Logger LOG = LoggerFactory.getLogger(DDLSemanticAnalyzer.class);
   private static final Map<Integer, String> TokenToTypeName = new HashMap<Integer, String>();
 
   // Equivalent to acidSinks, but for DDL operations that change data.
@@ -280,38 +263,6 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     case HiveParser.TOK_TRUNCATETABLE:
       analyzeTruncateTable(ast);
       break;
-    case HiveParser.TOK_DESCTABLE:
-      ctx.setResFile(ctx.getLocalTmpPath());
-      analyzeDescribeTable(ast);
-      break;
-    case HiveParser.TOK_SHOWTABLES:
-      ctx.setResFile(ctx.getLocalTmpPath());
-      analyzeShowTables(ast);
-      break;
-    case HiveParser.TOK_SHOW_TABLESTATUS:
-      ctx.setResFile(ctx.getLocalTmpPath());
-      analyzeShowTableStatus(ast);
-      break;
-    case HiveParser.TOK_SHOW_TBLPROPERTIES:
-      ctx.setResFile(ctx.getLocalTmpPath());
-      analyzeShowTableProperties(ast);
-      break;
-    case HiveParser.TOK_SHOWLOCKS:
-      ctx.setResFile(ctx.getLocalTmpPath());
-      analyzeShowLocks(ast);
-      break;
-    case HiveParser.TOK_SHOWDBLOCKS:
-      ctx.setResFile(ctx.getLocalTmpPath());
-      analyzeShowDbLocks(ast);
-      break;
-    case HiveParser.TOK_SHOWVIEWS:
-      ctx.setResFile(ctx.getLocalTmpPath());
-      analyzeShowViews(ast);
-      break;
-    case HiveParser.TOK_SHOWMATERIALIZEDVIEWS:
-      ctx.setResFile(ctx.getLocalTmpPath());
-      analyzeShowMaterializedViews(ast);
-      break;
     case HiveParser.TOK_ALTERVIEW: {
       final TableName tName = getQualifiedTableName((ASTNode) ast.getChild(0));
       ast = (ASTNode) ast.getChild(1);
@@ -324,12 +275,6 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       }
       break;
     }
-    case HiveParser.TOK_LOCKTABLE:
-      analyzeLockTable(ast);
-      break;
-    case HiveParser.TOK_UNLOCKTABLE:
-      analyzeUnlockTable(ast);
-      break;
    default:
       throw new SemanticException("Unsupported command: " + ast);
     }
@@ -1123,9 +1068,6 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
    */
   static class QualifiedNameUtil {
 
-    // delimiter to check DOT delimited qualified names
-    static final String delimiter = "\\.";
-
     /**
      * Get the fully qualified name in the ast. e.g. the ast of the form ^(DOT
      * ^(DOT a b) c) will generate a name of the form a.b.c
@@ -1148,495 +1090,6 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         return null;
       }
     }
-
-    // get the column path
-    // return column name if exists, column could be DOT separated.
-    // example: lintString.$elem$.myint
-    // return table name for column name if no column has been specified.
-    static public String getColPath(Hive db, ASTNode node, TableName tableName, Map<String, String> partSpec)
-        throws SemanticException {
-
-      // if this ast has only one child, then no column name specified.
-      if (node.getChildCount() == 1) {
-        return null;
-      }
-
-      ASTNode columnNode = null;
-      // Second child node could be partitionspec or column
-      if (node.getChildCount() > 1) {
-        if (partSpec == null) {
-          columnNode = (ASTNode) node.getChild(1);
-        } else {
-          columnNode = (ASTNode) node.getChild(2);
-        }
-      }
-
-      if (columnNode != null) {
-        return String.join(".", tableName.getNotEmptyDbTable(), QualifiedNameUtil.getFullyQualifiedName(columnNode));
-      } else {
-        return null;
-      }
-    }
-
-    // get partition metadata
-    static Map<String, String> getPartitionSpec(Hive db, ASTNode ast, TableName tableName)
-      throws SemanticException {
-      ASTNode partNode = null;
-      // if this ast has only one child, then no partition spec specified.
-      if (ast.getChildCount() == 1) {
-        return null;
-      }
-
-      // if ast has two children
-      // the 2nd child could be partition spec or columnName
-      // if the ast has 3 children, the second *has to* be partition spec
-      if (ast.getChildCount() > 2 && (((ASTNode) ast.getChild(1)).getType() != HiveParser.TOK_PARTSPEC)) {
-        throw new SemanticException(((ASTNode) ast.getChild(1)).getType() + " is not a partition specification");
-      }
-
-      if (((ASTNode) ast.getChild(1)).getType() == HiveParser.TOK_PARTSPEC) {
-        partNode = (ASTNode) ast.getChild(1);
-      }
-
-      if (partNode != null) {
-        Table tab = null;
-        try {
-          tab = db.getTable(tableName.getNotEmptyDbTable());
-        }
-        catch (InvalidTableException e) {
-          throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tableName.getNotEmptyDbTable()), e);
-        }
-        catch (HiveException e) {
-          throw new SemanticException(e.getMessage(), e);
-        }
-
-        Map<String, String> partSpec = null;
-        try {
-          partSpec = getValidatedPartSpec(tab, partNode, db.getConf(), false);
-        } catch (SemanticException e) {
-          // get exception in resolving partition
-          // it could be DESCRIBE table key
-          // return null
-          // continue processing for DESCRIBE table key
-          return null;
-        }
-
-        if (partSpec != null) {
-          Partition part = null;
-          try {
-            part = db.getPartition(tab, partSpec, false);
-          } catch (HiveException e) {
-            // if get exception in finding partition
-            // it could be DESCRIBE table key
-            // return null
-            // continue processing for DESCRIBE table key
-            return null;
-          }
-
-          // if partition is not found
-          // it is DESCRIBE table partition
-          // invalid partition exception
-          if (part == null) {
-            throw new SemanticException(ErrorMsg.INVALID_PARTITION.getMsg(partSpec.toString()));
-          }
-
-          // it is DESCRIBE table partition
-          // return partition metadata
-          return partSpec;
-        }
-      }
-
-      return null;
-    }
-
-  }
-
-  private void validateDatabase(String databaseName) throws SemanticException {
-    try {
-      if (!db.databaseExists(databaseName)) {
-        throw new SemanticException(ErrorMsg.DATABASE_NOT_EXISTS.getMsg(databaseName));
-      }
-    } catch (HiveException e) {
-      throw new SemanticException(ErrorMsg.DATABASE_NOT_EXISTS.getMsg(databaseName), e);
-    }
-  }
-
-  private void validateTable(TableName tableName, Map<String, String> partSpec)
-      throws SemanticException {
-    Table tab = getTable(tableName);
-    if (partSpec != null) {
-      PartitionUtils.getPartition(db, tab, partSpec, true);
-    }
-  }
-
-  /**
-   * A query like this will generate a tree as follows
-   *   "describe formatted default.maptable partition (b=100) id;"
-   * TOK_TABTYPE
-   *   TOK_TABNAME --> root for tablename, 2 child nodes mean DB specified
-   *     default
-   *     maptable
-   *   TOK_PARTSPEC  --> root node for partition spec. else columnName
-   *     TOK_PARTVAL
-   *       b
-   *       100
-   *   id           --> root node for columnName
-   * formatted
-   */
-  private void analyzeDescribeTable(ASTNode ast) throws SemanticException {
-    ASTNode tableTypeExpr = (ASTNode) ast.getChild(0);
-
-    final TableName tableName;
-    String colPath   = null;
-    Map<String, String> partSpec = null;
-
-    ASTNode tableNode = null;
-
-    // process the first node to extract tablename
-    // tablename is either TABLENAME or DBNAME.TABLENAME if db is given
-    if (((ASTNode) tableTypeExpr.getChild(0)).getType() == HiveParser.TOK_TABNAME) {
-      tableNode = (ASTNode) tableTypeExpr.getChild(0);
-      if (tableNode.getChildCount() == 1) {
-        tableName = HiveTableName.of(((ASTNode) tableNode.getChild(0)).getText());
-      } else {
-        tableName = TableName.fromString(((ASTNode) tableNode.getChild(1)).getText(),
-            SessionState.get().getCurrentCatalog(), ((ASTNode) tableNode.getChild(0)).getText());
-      }
-    } else {
-      throw new SemanticException(((ASTNode) tableTypeExpr.getChild(0)).getText() + " is not an expected token type");
-    }
-
-    // process the second child,if exists, node to get partition spec(s)
-    partSpec = QualifiedNameUtil.getPartitionSpec(db, tableTypeExpr, tableName);
-
-    // process the third child node,if exists, to get partition spec(s)
-    colPath  = QualifiedNameUtil.getColPath(db, tableTypeExpr, tableName, partSpec);
-
-    // if database is not the one currently using
-    // validate database
-    if (tableName.getDb() != null) {
-      validateDatabase(tableName.getDb());
-    }
-    if (partSpec != null) {
-      validateTable(tableName, partSpec);
-    }
-
-    boolean showColStats = false;
-    boolean isFormatted = false;
-    boolean isExt = false;
-    if (ast.getChildCount() == 2) {
-      int descOptions = ast.getChild(1).getType();
-      isFormatted = descOptions == HiveParser.KW_FORMATTED;
-      isExt = descOptions == HiveParser.KW_EXTENDED;
-      // in case of "DESCRIBE FORMATTED tablename column_name" statement, colPath
-      // will contain tablename.column_name. If column_name is not specified
-      // colPath will be equal to tableName. This is how we can differentiate
-      // if we are describing a table or column
-      if (colPath != null && isFormatted) {
-        showColStats = true;
-      }
-    }
-
-    inputs.add(new ReadEntity(getTable(tableName)));
-
-    DescTableDesc descTblDesc = new DescTableDesc(ctx.getResFile(), tableName, partSpec, colPath, isExt, isFormatted);
-    Task<?> ddlTask = TaskFactory.get(new DDLWork(getInputs(), getOutputs(), descTblDesc));
-    rootTasks.add(ddlTask);
-    String schema = showColStats ? DescTableDesc.COLUMN_STATISTICS_SCHEMA : DescTableDesc.SCHEMA;
-    setFetchTask(createFetchTask(schema));
-    LOG.info("analyzeDescribeTable done");
-  }
-
-  private void analyzeShowTables(ASTNode ast) throws SemanticException {
-    ShowTablesDesc showTblsDesc;
-    String dbName = SessionState.get().getCurrentDatabase();
-    String tableNames = null;
-    TableType tableTypeFilter = null;
-    boolean isExtended = false;
-
-    if (ast.getChildCount() > 4) {
-      throw new SemanticException(ErrorMsg.INVALID_AST_TREE.getMsg(ast.toStringTree()));
-    }
-
-    for (int i = 0; i < ast.getChildCount(); i++) {
-      ASTNode child = (ASTNode) ast.getChild(i);
-      if (child.getType() == HiveParser.TOK_FROM) { // Specifies a DB
-        dbName = unescapeIdentifier(ast.getChild(++i).getText());
-        validateDatabase(dbName);
-      } else if (child.getType() == HiveParser.TOK_TABLE_TYPE) { // Filter on table type
-        String tableType = unescapeIdentifier(child.getChild(0).getText());
-        if (!tableType.equalsIgnoreCase("table_type")) {
-          throw new SemanticException("SHOW TABLES statement only allows equality filter on table_type value");
-        }
-        tableTypeFilter = TableType.valueOf(unescapeSQLString(child.getChild(1).getText()));
-      } else if (child.getType() == HiveParser.KW_EXTENDED) { // Include table type
-        isExtended = true;
-      } else { // Uses a pattern
-        tableNames = unescapeSQLString(child.getText());
-      }
-    }
-
-    showTblsDesc = new ShowTablesDesc(ctx.getResFile(), dbName, tableNames, tableTypeFilter, isExtended);
-    inputs.add(new ReadEntity(getDatabase(dbName)));
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), showTblsDesc)));
-    setFetchTask(createFetchTask(showTblsDesc.getSchema()));
-  }
-
-  private void analyzeShowTableStatus(ASTNode ast) throws SemanticException {
-    ShowTableStatusDesc showTblStatusDesc;
-    String tableNames = getUnescapedName((ASTNode) ast.getChild(0));
-    String dbName = SessionState.get().getCurrentDatabase();
-    int children = ast.getChildCount();
-    Map<String, String> partSpec = null;
-    if (children >= 2) {
-      if (children > 3) {
-        throw new SemanticException(ErrorMsg.INVALID_AST_TREE.getMsg());
-      }
-      for (int i = 1; i < children; i++) {
-        ASTNode child = (ASTNode) ast.getChild(i);
-        if (child.getToken().getType() == HiveParser.Identifier) {
-          dbName = unescapeIdentifier(child.getText());
-        } else if (child.getToken().getType() == HiveParser.TOK_PARTSPEC) {
-          partSpec = getValidatedPartSpec(getTable(tableNames), child, conf, false);
-        } else {
-          throw new SemanticException(ErrorMsg.INVALID_AST_TREE.getMsg(child.toStringTree() +
-            " , Invalid token " + child.getToken().getType()));
-        }
-      }
-    }
-
-    if (partSpec != null) {
-      validateTable(HiveTableName.ofNullableWithNoDefault(tableNames), partSpec);
-    }
-
-    showTblStatusDesc = new ShowTableStatusDesc(ctx.getResFile().toString(), dbName, tableNames, partSpec);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), showTblStatusDesc)));
-    setFetchTask(createFetchTask(ShowTableStatusDesc.SCHEMA));
-  }
-
-  private void analyzeShowTableProperties(ASTNode ast) throws SemanticException {
-    ShowTablePropertiesDesc showTblPropertiesDesc;
-    TableName qualified = getQualifiedTableName((ASTNode) ast.getChild(0));
-    String propertyName = null;
-    if (ast.getChildCount() > 1) {
-      propertyName = unescapeSQLString(ast.getChild(1).getText());
-    }
-
-    validateTable(qualified, null);
-
-    showTblPropertiesDesc = new ShowTablePropertiesDesc(ctx.getResFile().toString(), qualified, propertyName);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), showTblPropertiesDesc)));
-    setFetchTask(createFetchTask(ShowTablePropertiesDesc.SCHEMA));
-  }
-
-  /**
-   * Add the task according to the parsed command tree. This is used for the CLI
-   * command "SHOW LOCKS;".
-   *
-   * @param ast
-   *          The parsed command tree.
-   * @throws SemanticException
-   *           Parsing failed
-   */
-  private void analyzeShowLocks(ASTNode ast) throws SemanticException {
-    String tableName = null;
-    Map<String, String> partSpec = null;
-    boolean isExtended = false;
-
-    if (ast.getChildCount() >= 1) {
-      // table for which show locks is being executed
-      for (int i = 0; i < ast.getChildCount(); i++) {
-        ASTNode child = (ASTNode) ast.getChild(i);
-        if (child.getType() == HiveParser.TOK_TABTYPE) {
-          ASTNode tableTypeExpr = child;
-          tableName =
-            QualifiedNameUtil.getFullyQualifiedName((ASTNode) tableTypeExpr.getChild(0));
-          // get partition metadata if partition specified
-          if (tableTypeExpr.getChildCount() == 2) {
-            ASTNode partSpecNode = (ASTNode) tableTypeExpr.getChild(1);
-            partSpec = getValidatedPartSpec(getTable(tableName), partSpecNode, conf, false);
-          }
-        } else if (child.getType() == HiveParser.KW_EXTENDED) {
-          isExtended = true;
-        }
-      }
-    }
-
-    HiveTxnManager txnManager = null;
-    try {
-      txnManager = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
-    } catch (LockException e) {
-      throw new SemanticException(e.getMessage());
-    }
-
-    ShowLocksDesc showLocksDesc = new ShowLocksDesc(ctx.getResFile(), tableName,
-        partSpec, isExtended, txnManager.useNewShowLocksFormat());
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), showLocksDesc)));
-    setFetchTask(createFetchTask(showLocksDesc.getSchema()));
-
-    // Need to initialize the lock manager
-    ctx.setNeedLockMgr(true);
-  }
-
-   /**
-    * Add the task according to the parsed command tree. This is used for the CLI
-   * command "SHOW LOCKS DATABASE database [extended];".
-   *
-   * @param ast
-   *          The parsed command tree.
-   * @throws SemanticException
-   *           Parsing failed
-   */
-  private void analyzeShowDbLocks(ASTNode ast) throws SemanticException {
-    boolean isExtended = (ast.getChildCount() > 1);
-    String dbName = stripQuotes(ast.getChild(0).getText());
-
-    HiveTxnManager txnManager = null;
-    try {
-      txnManager = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
-    } catch (LockException e) {
-      throw new SemanticException(e.getMessage());
-    }
-
-    ShowLocksDesc showLocksDesc = new ShowLocksDesc(ctx.getResFile(), dbName,
-                                                    isExtended, txnManager.useNewShowLocksFormat());
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), showLocksDesc)));
-    setFetchTask(createFetchTask(showLocksDesc.getSchema()));
-
-    // Need to initialize the lock manager
-    ctx.setNeedLockMgr(true);
-  }
-
-  private void analyzeShowViews(ASTNode ast) throws SemanticException {
-    ShowTablesDesc showViewsDesc;
-    String dbName = SessionState.get().getCurrentDatabase();
-    String viewNames = null;
-
-    if (ast.getChildCount() > 3) {
-      throw new SemanticException(ErrorMsg.GENERIC_ERROR.getMsg());
-    }
-
-    switch (ast.getChildCount()) {
-    case 1: // Uses a pattern
-      viewNames = unescapeSQLString(ast.getChild(0).getText());
-      showViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName, viewNames, TableType.VIRTUAL_VIEW);
-      break;
-    case 2: // Specifies a DB
-      assert (ast.getChild(0).getType() == HiveParser.TOK_FROM);
-      dbName = unescapeIdentifier(ast.getChild(1).getText());
-      validateDatabase(dbName);
-      showViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName, TableType.VIRTUAL_VIEW);
-      break;
-    case 3: // Uses a pattern and specifies a DB
-      assert (ast.getChild(0).getType() == HiveParser.TOK_FROM);
-      dbName = unescapeIdentifier(ast.getChild(1).getText());
-      viewNames = unescapeSQLString(ast.getChild(2).getText());
-      validateDatabase(dbName);
-      showViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName, viewNames, TableType.VIRTUAL_VIEW);
-      break;
-    default: // No pattern or DB
-      showViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName, TableType.VIRTUAL_VIEW);
-      break;
-    }
-
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), showViewsDesc)));
-    setFetchTask(createFetchTask(showViewsDesc.getSchema()));
-  }
-
-  private void analyzeShowMaterializedViews(ASTNode ast) throws SemanticException {
-    ShowTablesDesc showMaterializedViewsDesc;
-    String dbName = SessionState.get().getCurrentDatabase();
-    String materializedViewNames = null;
-
-    if (ast.getChildCount() > 3) {
-      throw new SemanticException(ErrorMsg.GENERIC_ERROR.getMsg());
-    }
-
-    switch (ast.getChildCount()) {
-    case 1: // Uses a pattern
-      materializedViewNames = unescapeSQLString(ast.getChild(0).getText());
-      showMaterializedViewsDesc = new ShowTablesDesc(
-          ctx.getResFile(), dbName, materializedViewNames, TableType.MATERIALIZED_VIEW);
-      break;
-    case 2: // Specifies a DB
-      assert (ast.getChild(0).getType() == HiveParser.TOK_FROM);
-      dbName = unescapeIdentifier(ast.getChild(1).getText());
-      validateDatabase(dbName);
-      showMaterializedViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName, TableType.MATERIALIZED_VIEW);
-      break;
-    case 3: // Uses a pattern and specifies a DB
-      assert (ast.getChild(0).getType() == HiveParser.TOK_FROM);
-      dbName = unescapeIdentifier(ast.getChild(1).getText());
-      materializedViewNames = unescapeSQLString(ast.getChild(2).getText());
-      validateDatabase(dbName);
-      showMaterializedViewsDesc = new ShowTablesDesc(
-          ctx.getResFile(), dbName, materializedViewNames, TableType.MATERIALIZED_VIEW);
-      break;
-    default: // No pattern or DB
-      showMaterializedViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName, TableType.MATERIALIZED_VIEW);
-      break;
-    }
-
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), showMaterializedViewsDesc)));
-    setFetchTask(createFetchTask(showMaterializedViewsDesc.getSchema()));
-  }
-
-  /**
-   * Add the task according to the parsed command tree. This is used for the CLI
-   * command "LOCK TABLE ..;".
-   *
-   * @param ast
-   *          The parsed command tree.
-   * @throws SemanticException
-   *           Parsing failed
-   */
-  private void analyzeLockTable(ASTNode ast)
-      throws SemanticException {
-    String tableName = getUnescapedName((ASTNode) ast.getChild(0)).toLowerCase();
-    String mode = unescapeIdentifier(ast.getChild(1).getText().toUpperCase());
-    List<Map<String, String>> partSpecs = getPartitionSpecs(getTable(tableName), ast);
-
-    // We only can have a single partition spec
-    assert (partSpecs.size() <= 1);
-    Map<String, String> partSpec = null;
-    if (partSpecs.size() > 0) {
-      partSpec = partSpecs.get(0);
-    }
-
-    LockTableDesc lockTblDesc = new LockTableDesc(tableName, mode, partSpec,
-        HiveConf.getVar(conf, ConfVars.HIVEQUERYID), ctx.getCmd());
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), lockTblDesc)));
-
-    // Need to initialize the lock manager
-    ctx.setNeedLockMgr(true);
-  }
-
-  /**
-   * Add the task according to the parsed command tree. This is used for the CLI
-   * command "UNLOCK TABLE ..;".
-   *
-   * @param ast
-   *          The parsed command tree.
-   * @throws SemanticException
-   *           Parsing failed
-   */
-  private void analyzeUnlockTable(ASTNode ast)
-      throws SemanticException {
-    String tableName = getUnescapedName((ASTNode) ast.getChild(0));
-    List<Map<String, String>> partSpecs = getPartitionSpecs(getTable(tableName), ast);
-
-    // We only can have a single partition spec
-    assert (partSpecs.size() <= 1);
-    Map<String, String> partSpec = null;
-    if (partSpecs.size() > 0) {
-      partSpec = partSpecs.get(0);
-    }
-
-    UnlockTableDesc unlockTblDesc = new UnlockTableDesc(tableName, partSpec);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), unlockTblDesc)));
-
-    // Need to initialize the lock manager
-    ctx.setNeedLockMgr(true);
   }
 
   private void analyzeAlterTableRename(TableName source, ASTNode ast, boolean expectView)
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
index 8d1136a..9748fbd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
@@ -103,18 +103,9 @@ public final class SemanticAnalyzerFactory {
         queryState.setCommandType(HiveOperation.ALTERVIEW_AS);
         return new SemanticAnalyzer(queryState);
       }
-      case HiveParser.TOK_DESCTABLE:
       case HiveParser.TOK_MSCK:
-      case HiveParser.TOK_SHOWTABLES:
-      case HiveParser.TOK_SHOW_TABLESTATUS:
-      case HiveParser.TOK_SHOW_TBLPROPERTIES:
-      case HiveParser.TOK_SHOWLOCKS:
       case HiveParser.TOK_SHOWDBLOCKS:
       case HiveParser.TOK_SHOWCONF:
-      case HiveParser.TOK_SHOWVIEWS:
-      case HiveParser.TOK_SHOWMATERIALIZEDVIEWS:
-      case HiveParser.TOK_LOCKTABLE:
-      case HiveParser.TOK_UNLOCKTABLE:
       case HiveParser.TOK_TRUNCATETABLE:
         return new DDLSemanticAnalyzer(queryState);
 
diff --git a/ql/src/test/queries/clientpositive/show_materialized_views.q b/ql/src/test/queries/clientpositive/show_materialized_views.q
index c6ae021..81f86a7 100644
--- a/ql/src/test/queries/clientpositive/show_materialized_views.q
+++ b/ql/src/test/queries/clientpositive/show_materialized_views.q
@@ -29,9 +29,12 @@ CREATE MATERIALIZED VIEW shtb_test2_view2 DISABLE REWRITE AS
 SELECT * FROM shtb_test2 where KEY > 100 and KEY < 200;
 
 USE test1;
+EXPLAIN SHOW MATERIALIZED VIEWS;
 SHOW MATERIALIZED VIEWS;
+EXPLAIN SHOW MATERIALIZED VIEWS '*test*';
 SHOW MATERIALIZED VIEWS '*test*';
 SHOW MATERIALIZED VIEWS '*view2';
+EXPLAIN SHOW MATERIALIZED VIEWS LIKE 'shtb_test1_view1|shtb_test1_view2';
 SHOW MATERIALIZED VIEWS LIKE 'shtb_test1_view1|shtb_test1_view2';
 
 USE test2;
@@ -39,10 +42,13 @@ SHOW MATERIALIZED VIEWS 'shtb_*';
 
 -- SHOW MATERIALIZED VIEWS basic syntax tests
 USE default;
+EXPLAIN SHOW MATERIALIZED VIEWS FROM test1;
 SHOW MATERIALIZED VIEWS FROM test1;
 SHOW MATERIALIZED VIEWS FROM test2;
+EXPLAIN SHOW MATERIALIZED VIEWS IN test1;
 SHOW MATERIALIZED VIEWS IN test1;
 SHOW MATERIALIZED VIEWS IN default;
+EXPLAIN SHOW MATERIALIZED VIEWS IN test1 "shtb_test*";
 SHOW MATERIALIZED VIEWS IN test1 "shtb_test*";
 DESCRIBE FORMATTED test1.shtb_full_view2;
 DESCRIBE FORMATTED test1.shtb_test1_view1;
diff --git a/ql/src/test/queries/clientpositive/show_tables.q b/ql/src/test/queries/clientpositive/show_tables.q
index 087d40f..8576daa 100644
--- a/ql/src/test/queries/clientpositive/show_tables.q
+++ b/ql/src/test/queries/clientpositive/show_tables.q
@@ -21,13 +21,21 @@ CREATE VIEW test_view_n100 AS SELECT * FROM foo_n4;
 
 -- SHOW TABLES basic syntax tests
 USE default;
+EXPLAIN SHOW TABLES FROM test_db;
 SHOW TABLES FROM test_db;
+EXPLAIN SHOW EXTENDED TABLES FROM test_db;
 SHOW EXTENDED TABLES FROM test_db;
+EXPLAIN SHOW TABLES IN test_db;
 SHOW TABLES IN test_db;
+EXPLAIN SHOW EXTENDED TABLES IN test_db;
 SHOW EXTENDED TABLES IN test_db;
+EXPLAIN SHOW TABLES IN test_db "test*";
 SHOW TABLES IN test_db "test*";
+EXPLAIN SHOW TABLES IN test_db LIKE "nomatch";
 SHOW TABLES IN test_db LIKE "nomatch";
+EXPLAIN SHOW TABLES IN test_db WHERE `table_type` = "MANAGED_TABLE";
 SHOW TABLES IN test_db WHERE `table_type` = "MANAGED_TABLE";
+EXPLAIN SHOW EXTENDED TABLES IN test_db WHERE `table_type` = "VIRTUAL_VIEW";
 SHOW EXTENDED TABLES IN test_db WHERE `table_type` = "VIRTUAL_VIEW";
 
 -- SHOW TABLE EXTENDED basic syntax tests and wildcard
diff --git a/ql/src/test/queries/clientpositive/show_views.q b/ql/src/test/queries/clientpositive/show_views.q
index 726ad4b..1af89b6 100644
--- a/ql/src/test/queries/clientpositive/show_views.q
+++ b/ql/src/test/queries/clientpositive/show_views.q
@@ -13,9 +13,12 @@ CREATE VIEW shtb_test1_view1_n0 AS SELECT * FROM shtb_test1_n1 where KEY > 1000
 CREATE VIEW shtb_test2_view2_n0 AS SELECT * FROM shtb_test2_n1 where KEY > 100 and KEY < 200;
 
 USE test1;
+EXPLAIN SHOW VIEWS;
 SHOW VIEWS;
+EXPLAIN SHOW VIEWS 'test_*';
 SHOW VIEWS 'test_*';
 SHOW VIEWS '*view2';
+EXPLAIN SHOW VIEWS LIKE 'test_view1|test_view2';
 SHOW VIEWS LIKE 'test_view1|test_view2';
 
 USE test2;
@@ -23,10 +26,13 @@ SHOW VIEWS 'shtb_*';
 
 -- SHOW VIEWS basic syntax tests
 USE default;
+EXPLAIN SHOW VIEWS FROM test1;
 SHOW VIEWS FROM test1;
 SHOW VIEWS FROM test2;
+EXPLAIN SHOW VIEWS IN test1;
 SHOW VIEWS IN test1;
 SHOW VIEWS IN default;
+EXPLAIN SHOW VIEWS IN test1 "shtb_test_*";
 SHOW VIEWS IN test1 "shtb_test_*";
 SHOW VIEWS IN test2 LIKE "nomatch";
 
diff --git a/ql/src/test/results/clientpositive/show_materialized_views.q.out b/ql/src/test/results/clientpositive/show_materialized_views.q.out
index c332851..57bd93b 100644
--- a/ql/src/test/results/clientpositive/show_materialized_views.q.out
+++ b/ql/src/test/results/clientpositive/show_materialized_views.q.out
@@ -122,6 +122,25 @@ PREHOOK: Input: database:test1
 POSTHOOK: query: USE test1
 POSTHOOK: type: SWITCHDATABASE
 POSTHOOK: Input: database:test1
+PREHOOK: query: EXPLAIN SHOW MATERIALIZED VIEWS
+PREHOOK: type: SHOWMATERIALIZEDVIEWS
+POSTHOOK: query: EXPLAIN SHOW MATERIALIZED VIEWS
+POSTHOOK: type: SHOWMATERIALIZEDVIEWS
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Materialized Views
+      database name: test1
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW MATERIALIZED VIEWS
 PREHOOK: type: SHOWMATERIALIZEDVIEWS
 POSTHOOK: query: SHOW MATERIALIZED VIEWS
@@ -131,6 +150,26 @@ shtb_full_view2     	Yes                 	Manual refresh (Valid for 5min)
 shtb_test1_view1    	No                  	Manual refresh      
 shtb_test1_view2    	Yes                 	Manual refresh (Valid always)
 	 	 
+PREHOOK: query: EXPLAIN SHOW MATERIALIZED VIEWS '*test*'
+PREHOOK: type: SHOWMATERIALIZEDVIEWS
+POSTHOOK: query: EXPLAIN SHOW MATERIALIZED VIEWS '*test*'
+POSTHOOK: type: SHOWMATERIALIZEDVIEWS
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Materialized Views
+      database name: test1
+      pattern: *test*
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW MATERIALIZED VIEWS '*test*'
 PREHOOK: type: SHOWMATERIALIZEDVIEWS
 POSTHOOK: query: SHOW MATERIALIZED VIEWS '*test*'
@@ -147,6 +186,26 @@ POSTHOOK: type: SHOWMATERIALIZEDVIEWS
 shtb_full_view2     	Yes                 	Manual refresh (Valid for 5min)
 shtb_test1_view2    	Yes                 	Manual refresh (Valid always)
 	 	 
+PREHOOK: query: EXPLAIN SHOW MATERIALIZED VIEWS LIKE 'shtb_test1_view1|shtb_test1_view2'
+PREHOOK: type: SHOWMATERIALIZEDVIEWS
+POSTHOOK: query: EXPLAIN SHOW MATERIALIZED VIEWS LIKE 'shtb_test1_view1|shtb_test1_view2'
+POSTHOOK: type: SHOWMATERIALIZEDVIEWS
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Materialized Views
+      database name: test1
+      pattern: shtb_test1_view1|shtb_test1_view2
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW MATERIALIZED VIEWS LIKE 'shtb_test1_view1|shtb_test1_view2'
 PREHOOK: type: SHOWMATERIALIZEDVIEWS
 POSTHOOK: query: SHOW MATERIALIZED VIEWS LIKE 'shtb_test1_view1|shtb_test1_view2'
@@ -175,6 +234,25 @@ PREHOOK: Input: database:default
 POSTHOOK: query: USE default
 POSTHOOK: type: SWITCHDATABASE
 POSTHOOK: Input: database:default
+PREHOOK: query: EXPLAIN SHOW MATERIALIZED VIEWS FROM test1
+PREHOOK: type: SHOWMATERIALIZEDVIEWS
+POSTHOOK: query: EXPLAIN SHOW MATERIALIZED VIEWS FROM test1
+POSTHOOK: type: SHOWMATERIALIZEDVIEWS
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Materialized Views
+      database name: test1
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW MATERIALIZED VIEWS FROM test1
 PREHOOK: type: SHOWMATERIALIZEDVIEWS
 POSTHOOK: query: SHOW MATERIALIZED VIEWS FROM test1
@@ -192,6 +270,25 @@ POSTHOOK: type: SHOWMATERIALIZEDVIEWS
 shtb_test1_view1    	No                  	Manual refresh      
 shtb_test2_view2    	No                  	Manual refresh      
 	 	 
+PREHOOK: query: EXPLAIN SHOW MATERIALIZED VIEWS IN test1
+PREHOOK: type: SHOWMATERIALIZEDVIEWS
+POSTHOOK: query: EXPLAIN SHOW MATERIALIZED VIEWS IN test1
+POSTHOOK: type: SHOWMATERIALIZEDVIEWS
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Materialized Views
+      database name: test1
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW MATERIALIZED VIEWS IN test1
 PREHOOK: type: SHOWMATERIALIZEDVIEWS
 POSTHOOK: query: SHOW MATERIALIZED VIEWS IN test1
@@ -205,6 +302,26 @@ PREHOOK: query: SHOW MATERIALIZED VIEWS IN default
 PREHOOK: type: SHOWMATERIALIZEDVIEWS
 POSTHOOK: query: SHOW MATERIALIZED VIEWS IN default
 POSTHOOK: type: SHOWMATERIALIZEDVIEWS
+PREHOOK: query: EXPLAIN SHOW MATERIALIZED VIEWS IN test1 "shtb_test*"
+PREHOOK: type: SHOWMATERIALIZEDVIEWS
+POSTHOOK: query: EXPLAIN SHOW MATERIALIZED VIEWS IN test1 "shtb_test*"
+POSTHOOK: type: SHOWMATERIALIZEDVIEWS
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Materialized Views
+      database name: test1
+      pattern: shtb_test*
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW MATERIALIZED VIEWS IN test1 "shtb_test*"
 PREHOOK: type: SHOWMATERIALIZEDVIEWS
 POSTHOOK: query: SHOW MATERIALIZED VIEWS IN test1 "shtb_test*"
diff --git a/ql/src/test/results/clientpositive/show_tables.q.out b/ql/src/test/results/clientpositive/show_tables.q.out
index 82cc06f..342ebd4 100644
--- a/ql/src/test/results/clientpositive/show_tables.q.out
+++ b/ql/src/test/results/clientpositive/show_tables.q.out
@@ -131,6 +131,27 @@ PREHOOK: Input: database:default
 POSTHOOK: query: USE default
 POSTHOOK: type: SWITCHDATABASE
 POSTHOOK: Input: database:default
+PREHOOK: query: EXPLAIN SHOW TABLES FROM test_db
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:test_db
+POSTHOOK: query: EXPLAIN SHOW TABLES FROM test_db
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:test_db
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Tables
+      database name: test_db
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW TABLES FROM test_db
 PREHOOK: type: SHOWTABLES
 PREHOOK: Input: database:test_db
@@ -141,6 +162,28 @@ bar_n0
 baz
 foo_n4
 test_view_n100
+PREHOOK: query: EXPLAIN SHOW EXTENDED TABLES FROM test_db
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:test_db
+POSTHOOK: query: EXPLAIN SHOW EXTENDED TABLES FROM test_db
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:test_db
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Tables
+      database name: test_db
+      extended: true
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW EXTENDED TABLES FROM test_db
 PREHOOK: type: SHOWTABLES
 PREHOOK: Input: database:test_db
@@ -153,6 +196,27 @@ baz                 	MANAGED_TABLE
 foo_n4              	MANAGED_TABLE       
 test_view_n100      	VIRTUAL_VIEW        
 	 
+PREHOOK: query: EXPLAIN SHOW TABLES IN test_db
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:test_db
+POSTHOOK: query: EXPLAIN SHOW TABLES IN test_db
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:test_db
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Tables
+      database name: test_db
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW TABLES IN test_db
 PREHOOK: type: SHOWTABLES
 PREHOOK: Input: database:test_db
@@ -163,6 +227,28 @@ bar_n0
 baz
 foo_n4
 test_view_n100
+PREHOOK: query: EXPLAIN SHOW EXTENDED TABLES IN test_db
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:test_db
+POSTHOOK: query: EXPLAIN SHOW EXTENDED TABLES IN test_db
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:test_db
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Tables
+      database name: test_db
+      extended: true
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW EXTENDED TABLES IN test_db
 PREHOOK: type: SHOWTABLES
 PREHOOK: Input: database:test_db
@@ -175,6 +261,28 @@ baz                 	MANAGED_TABLE
 foo_n4              	MANAGED_TABLE       
 test_view_n100      	VIRTUAL_VIEW        
 	 
+PREHOOK: query: EXPLAIN SHOW TABLES IN test_db "test*"
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:test_db
+POSTHOOK: query: EXPLAIN SHOW TABLES IN test_db "test*"
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:test_db
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Tables
+      database name: test_db
+      pattern: test*
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW TABLES IN test_db "test*"
 PREHOOK: type: SHOWTABLES
 PREHOOK: Input: database:test_db
@@ -182,12 +290,56 @@ POSTHOOK: query: SHOW TABLES IN test_db "test*"
 POSTHOOK: type: SHOWTABLES
 POSTHOOK: Input: database:test_db
 test_view_n100
+PREHOOK: query: EXPLAIN SHOW TABLES IN test_db LIKE "nomatch"
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:test_db
+POSTHOOK: query: EXPLAIN SHOW TABLES IN test_db LIKE "nomatch"
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:test_db
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Tables
+      database name: test_db
+      pattern: nomatch
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW TABLES IN test_db LIKE "nomatch"
 PREHOOK: type: SHOWTABLES
 PREHOOK: Input: database:test_db
 POSTHOOK: query: SHOW TABLES IN test_db LIKE "nomatch"
 POSTHOOK: type: SHOWTABLES
 POSTHOOK: Input: database:test_db
+PREHOOK: query: EXPLAIN SHOW TABLES IN test_db WHERE `table_type` = "MANAGED_TABLE"
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:test_db
+POSTHOOK: query: EXPLAIN SHOW TABLES IN test_db WHERE `table_type` = "MANAGED_TABLE"
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:test_db
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Tables
+      database name: test_db
+      table type filter: MANAGED_TABLE
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW TABLES IN test_db WHERE `table_type` = "MANAGED_TABLE"
 PREHOOK: type: SHOWTABLES
 PREHOOK: Input: database:test_db
@@ -197,6 +349,29 @@ POSTHOOK: Input: database:test_db
 bar_n0
 baz
 foo_n4
+PREHOOK: query: EXPLAIN SHOW EXTENDED TABLES IN test_db WHERE `table_type` = "VIRTUAL_VIEW"
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:test_db
+POSTHOOK: query: EXPLAIN SHOW EXTENDED TABLES IN test_db WHERE `table_type` = "VIRTUAL_VIEW"
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:test_db
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Tables
+      database name: test_db
+      table type filter: VIRTUAL_VIEW
+      extended: true
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW EXTENDED TABLES IN test_db WHERE `table_type` = "VIRTUAL_VIEW"
 PREHOOK: type: SHOWTABLES
 PREHOOK: Input: database:test_db
diff --git a/ql/src/test/results/clientpositive/show_views.q.out b/ql/src/test/results/clientpositive/show_views.q.out
index 311f700..b5d2027 100644
--- a/ql/src/test/results/clientpositive/show_views.q.out
+++ b/ql/src/test/results/clientpositive/show_views.q.out
@@ -117,6 +117,25 @@ PREHOOK: Input: database:test1
 POSTHOOK: query: USE test1
 POSTHOOK: type: SWITCHDATABASE
 POSTHOOK: Input: database:test1
+PREHOOK: query: EXPLAIN SHOW VIEWS
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: EXPLAIN SHOW VIEWS
+POSTHOOK: type: SHOWVIEWS
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Views
+      database name: test1
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW VIEWS
 PREHOOK: type: SHOWVIEWS
 POSTHOOK: query: SHOW VIEWS
@@ -124,6 +143,26 @@ POSTHOOK: type: SHOWVIEWS
 shtb_full_view2_n0
 shtb_test1_view1_n0
 shtb_test1_view2_n0
+PREHOOK: query: EXPLAIN SHOW VIEWS 'test_*'
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: EXPLAIN SHOW VIEWS 'test_*'
+POSTHOOK: type: SHOWVIEWS
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Views
+      database name: test1
+      pattern: test_*
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW VIEWS 'test_*'
 PREHOOK: type: SHOWVIEWS
 POSTHOOK: query: SHOW VIEWS 'test_*'
@@ -132,6 +171,26 @@ PREHOOK: query: SHOW VIEWS '*view2'
 PREHOOK: type: SHOWVIEWS
 POSTHOOK: query: SHOW VIEWS '*view2'
 POSTHOOK: type: SHOWVIEWS
+PREHOOK: query: EXPLAIN SHOW VIEWS LIKE 'test_view1|test_view2'
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: EXPLAIN SHOW VIEWS LIKE 'test_view1|test_view2'
+POSTHOOK: type: SHOWVIEWS
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Views
+      database name: test1
+      pattern: test_view1|test_view2
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW VIEWS LIKE 'test_view1|test_view2'
 PREHOOK: type: SHOWVIEWS
 POSTHOOK: query: SHOW VIEWS LIKE 'test_view1|test_view2'
@@ -154,6 +213,25 @@ PREHOOK: Input: database:default
 POSTHOOK: query: USE default
 POSTHOOK: type: SWITCHDATABASE
 POSTHOOK: Input: database:default
+PREHOOK: query: EXPLAIN SHOW VIEWS FROM test1
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: EXPLAIN SHOW VIEWS FROM test1
+POSTHOOK: type: SHOWVIEWS
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Views
+      database name: test1
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW VIEWS FROM test1
 PREHOOK: type: SHOWVIEWS
 POSTHOOK: query: SHOW VIEWS FROM test1
@@ -167,6 +245,25 @@ POSTHOOK: query: SHOW VIEWS FROM test2
 POSTHOOK: type: SHOWVIEWS
 shtb_test1_view1_n0
 shtb_test2_view2_n0
+PREHOOK: query: EXPLAIN SHOW VIEWS IN test1
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: EXPLAIN SHOW VIEWS IN test1
+POSTHOOK: type: SHOWVIEWS
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Views
+      database name: test1
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW VIEWS IN test1
 PREHOOK: type: SHOWVIEWS
 POSTHOOK: query: SHOW VIEWS IN test1
@@ -178,6 +275,26 @@ PREHOOK: query: SHOW VIEWS IN default
 PREHOOK: type: SHOWVIEWS
 POSTHOOK: query: SHOW VIEWS IN default
 POSTHOOK: type: SHOWVIEWS
+PREHOOK: query: EXPLAIN SHOW VIEWS IN test1 "shtb_test_*"
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: EXPLAIN SHOW VIEWS IN test1 "shtb_test_*"
+POSTHOOK: type: SHOWVIEWS
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+  Stage-1 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-0
+    Show Views
+      database name: test1
+      pattern: shtb_test_*
+
+  Stage: Stage-1
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
 PREHOOK: query: SHOW VIEWS IN test1 "shtb_test_*"
 PREHOOK: type: SHOWVIEWS
 POSTHOOK: query: SHOW VIEWS IN test1 "shtb_test_*"