You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by mi...@apache.org on 2023/12/06 18:17:38 UTC

(impala) branch master updated: IMPALA-3268: Add support for SHOW VIEWS statement

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 7a99449a5 IMPALA-3268: Add support for SHOW VIEWS statement
7a99449a5 is described below

commit 7a99449a540bd6bd24cce3ddf2f95b255be29606
Author: Fang-Yu Rao <fa...@cloudera.com>
AuthorDate: Mon Nov 6 11:41:58 2023 -0800

    IMPALA-3268: Add support for SHOW VIEWS statement
    
    This patch adds the support for the SHOW VIEWS statement and does not
    change the current behavior of the SHOW TABLES statement. It is
    recommended that we start the catalog server with
    '--pull_table_types_and_comments=true' when executing this command
    so that the table type of every Table is always available.
    
    Specifically, after this patch, a user will be able to retrieve from
    Impala the names of the Table's whose getTableType() evaluate to
    TImpalaTableType.VIEW that match an optional pattern.
    
    Similar to the SHOW TABLES statement, the following are supported
    after this patch.
    1. SHOW VIEWS.
    2. SHOW VIEWS "<P1>|<P2>|<P3>", where <PN> denotes the N-th pattern,
       "|" denotes choice and each pattern may contain wildcards denoted by
       "*".
    3. SHOW VIEWS IN <db>, where <db> denotes the name of a database.
    Refer to the added end-to-end test cases for further details.
    
    Recall that Hive distinguishes materialized views from views. To align
    Impala's behavior with that of Hive with respect to the SHOW VIEWS
    statement, in this patch we introduce
    TImpalaTableType.MATERIALIZED_VIEW as a table type that corresponds to
    the table type of 'MATERIALIZED_VIEW' for a
    org.apache.hadoop.hive.metastore.api.Table. Due to this we have to
    revise the mappings from Hive's table types to Impala's table types in
    various places and have to slightly modify the logic used to
    instantiate a table with an appropriate type.
    
    Testing:
     - Added various frontend and end-to-end tests to verify the behavior
       of the SHOW VIEWS statement.
    
    Change-Id: I321fc5350392a815949a4e7d2a64d60466689788
    Reviewed-on: http://gerrit.cloudera.org:8080/20669
    Reviewed-by: Quanlong Huang <hu...@gmail.com>
    Tested-by: Michael Smith <mi...@cloudera.com>
---
 be/src/service/client-request-state.cc             |  12 +-
 be/src/service/frontend.cc                         |   8 +
 be/src/service/frontend.h                          |  16 +-
 common/thrift/CatalogObjects.thrift                |   6 +
 common/thrift/CatalogService.thrift                |   6 +-
 common/thrift/Frontend.thrift                      |  15 +-
 fe/src/main/cup/sql-parser.cup                     |  18 ++-
 .../apache/impala/analysis/AnalysisContext.java    |  19 ++-
 ...wTablesStmt.java => ShowTablesOrViewsStmt.java} |  50 ++----
 .../org/apache/impala/analysis/ShowTablesStmt.java |  84 ++--------
 .../org/apache/impala/analysis/ShowViewsStmt.java  |  55 +++++++
 .../java/org/apache/impala/catalog/Catalog.java    |  17 +-
 fe/src/main/java/org/apache/impala/catalog/Db.java |  19 ++-
 .../java/org/apache/impala/catalog/FeCatalog.java  |   6 +-
 .../main/java/org/apache/impala/catalog/FeDb.java  |  10 +-
 .../impala/catalog/Hive3MetastoreShimBase.java     |   5 +-
 .../org/apache/impala/catalog/IncompleteTable.java |   2 +
 .../main/java/org/apache/impala/catalog/Table.java |  12 +-
 .../apache/impala/catalog/local/LocalCatalog.java  |  13 +-
 .../org/apache/impala/catalog/local/LocalDb.java   |  15 ++
 .../java/org/apache/impala/service/Frontend.java   |  24 ++-
 .../org/apache/impala/service/JniFrontend.java     |   3 +-
 fe/src/main/jflex/sql-scanner.flex                 |   1 +
 .../org/apache/impala/analysis/ParserTest.java     |   8 +
 .../java/org/apache/impala/analysis/ToSqlTest.java |  24 +++
 .../authorization/AuthorizationStmtTest.java       |  16 ++
 .../queries/QueryTest/show_views.test              | 176 +++++++++++++++++++++
 tests/custom_cluster/test_preload_table_types.py   |   3 +-
 tests/custom_cluster/test_show_views_statements.py |  44 ++++++
 29 files changed, 532 insertions(+), 155 deletions(-)

diff --git a/be/src/service/client-request-state.cc b/be/src/service/client-request-state.cc
index a4141a44f..0f24c0717 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -354,16 +354,18 @@ Status ClientRequestState::ExecLocalCatalogOp(
       session_->database = exec_request_->catalog_op_request.use_db_params.db;
       return Status::OK();
     }
-    case TCatalogOpType::SHOW_TABLES: {
+    case TCatalogOpType::SHOW_TABLES:
+    case TCatalogOpType::SHOW_VIEWS: {
       const TShowTablesParams* params = &catalog_op.show_tables_params;
-      // A NULL pattern means match all tables. However, Thrift string types can't
-      // be NULL in C++, so we have to test if it's set rather than just blindly
-      // using the value.
+      // A NULL pattern means match all tables of the specified table types. However,
+      // Thrift string types can't be NULL in C++, so we have to test if it's set rather
+      // than just blindly using the value.
       const string* table_name =
           params->__isset.show_pattern ? &(params->show_pattern) : NULL;
       TGetTablesResult table_names;
+      const set<TImpalaTableType::type>& table_types = params->table_types;
       RETURN_IF_ERROR(frontend_->GetTableNames(params->db, table_name,
-          &query_ctx_.session, &table_names));
+          &query_ctx_.session, table_types, &table_names));
       SetResultSet(table_names.tables);
       return Status::OK();
     }
diff --git a/be/src/service/frontend.cc b/be/src/service/frontend.cc
index 65f830e88..40f23ce24 100644
--- a/be/src/service/frontend.cc
+++ b/be/src/service/frontend.cc
@@ -198,8 +198,16 @@ Status Frontend::GetCatalogMetrics(TGetCatalogMetricsResult* resp) {
 
 Status Frontend::GetTableNames(const string& db, const string* pattern,
     const TSessionState* session, TGetTablesResult* table_names) {
+    set<TImpalaTableType::type> table_types = set<TImpalaTableType::type>();
+    return GetTableNames(db, pattern, session, table_types, table_names);
+}
+
+Status Frontend::GetTableNames(const string& db, const string* pattern,
+    const TSessionState* session, const set<TImpalaTableType::type>& table_types,
+    TGetTablesResult* table_names) {
   TGetTablesParams params;
   params.__set_db(db);
+  params.__set_table_types(table_types);
   if (pattern != NULL) params.__set_pattern(*pattern);
   if (session != NULL) params.__set_session(*session);
   return JniUtil::CallJniMethod(fe_, get_table_names_id_, params, table_names);
diff --git a/be/src/service/frontend.h b/be/src/service/frontend.h
index d450b0945..363d2d519 100644
--- a/be/src/service/frontend.h
+++ b/be/src/service/frontend.h
@@ -21,6 +21,7 @@
 #include <jni.h>
 
 #include "common/status.h"
+#include "gen-cpp/CatalogService_types.h"
 #include "gen-cpp/Frontend_types.h"
 #include "gen-cpp/ImpalaHiveServer2Service.h"
 #include "gen-cpp/ImpalaService.h"
@@ -59,8 +60,16 @@ class Frontend {
   /// Get the metrics from the catalog used by this frontend.
   Status GetCatalogMetrics(TGetCatalogMetricsResult* resp);
 
-  /// Returns all matching table names, per Hive's "SHOW TABLES <pattern>". Each
-  /// table name returned is unqualified.
+  /// Get all matching table names, per Hive's "SHOW TABLES <pattern>" regardless of the
+  /// table type.
+  Status GetTableNames(const std::string& db, const std::string* pattern,
+      const TSessionState* session, TGetTablesResult* table_names);
+
+  /// Returns all matching table names, per Hive's "SHOW TABLES <pattern>" such that each
+  /// corresponds to a table whose type is in table_types for a non-empty table_types.
+  /// Each table name returned is unqualified.
+  /// If table_types is empty, then all types of tables will be considered when their
+  /// table names are matched against the pattern.
   /// If pattern is NULL, match all tables otherwise match only those tables that
   /// match the pattern string. Patterns are "p1|p2|p3" where | denotes choice,
   /// and each pN may contain wildcards denoted by '*' which match all strings.
@@ -70,7 +79,8 @@ class Frontend {
   /// the session should be set to NULL which will skip privilege checks returning all
   /// results.
   Status GetTableNames(const std::string& db, const std::string* pattern,
-      const TSessionState* session, TGetTablesResult* table_names);
+      const TSessionState* session, const std::set<TImpalaTableType::type>& table_types,
+      TGetTablesResult* table_names);
 
   /// Return all databases matching the optional argument 'pattern'.
   /// If pattern is NULL, match all databases otherwise match only those databases that
diff --git a/common/thrift/CatalogObjects.thrift b/common/thrift/CatalogObjects.thrift
index 116e484cf..824c9fe17 100644
--- a/common/thrift/CatalogObjects.thrift
+++ b/common/thrift/CatalogObjects.thrift
@@ -56,6 +56,12 @@ enum TTableType {
   // Type for tables that we haven't loaded its full metadata so we don't know whether
   // it's a HDFS or Kudu table, etc. We just know it's not a view.
   UNLOADED_TABLE = 6
+  // We added MATERIALIZED_VIEW as a table type to TImpalaTableType in IMPALA-3268.
+  // To properly set the table type of a materialized view when calling
+  // JniFrontend#updateCatalogCache(), we need to also introduce this table type here
+  // so that a materialized view will not be classified as a table. Refer to
+  // IncompleteTable#toThrift() for further details.
+  MATERIALIZED_VIEW = 7
 }
 
 // TODO: Separate the storage engines (e.g. Kudu) from the file formats.
diff --git a/common/thrift/CatalogService.thrift b/common/thrift/CatalogService.thrift
index 1f915393f..7d2282d3f 100644
--- a/common/thrift/CatalogService.thrift
+++ b/common/thrift/CatalogService.thrift
@@ -523,12 +523,12 @@ struct TPartialTableInfo {
   13: optional CatalogObjects.TIcebergTable iceberg_table
 }
 
-// Table types in the user's perspective. Though we treat materialized view as table
-// internally, materialized views are shown in view type to the users.
+// Table types in the user's perspective.
 enum TImpalaTableType {
   TABLE,
   VIEW,
-  UNKNOWN
+  UNKNOWN,
+  MATERIALIZED_VIEW
 }
 
 struct TBriefTableMeta {
diff --git a/common/thrift/Frontend.thrift b/common/thrift/Frontend.thrift
index f81034bd7..e7118ed19 100644
--- a/common/thrift/Frontend.thrift
+++ b/common/thrift/Frontend.thrift
@@ -64,8 +64,8 @@ struct THiveUdfExecutorCtorParams {
   7: required i64 output_buffer_ptr
 }
 
-// Arguments to getTableNames, which returns a list of tables that match an
-// optional pattern.
+// Arguments to getTableNames, which returns a list of tables that are of specified table
+// types and match an optional pattern.
 struct TGetTablesParams {
   // If not set, match tables in all DBs
   1: optional string db
@@ -77,6 +77,10 @@ struct TGetTablesParams {
   // enabled, only the tables this user has access to will be returned. If not
   // set, access checks will be skipped (used for internal Impala requests)
   3: optional Query.TSessionState session
+
+  // This specifies the types of tables that should be returned. If not set, all types of
+  // tables are considered when their names are matched against pattern.
+  4: optional set<CatalogService.TImpalaTableType> table_types = []
 }
 
 // getTableNames returns a list of unqualified table names
@@ -246,7 +250,7 @@ struct TShowFunctionsParams {
   3: optional string show_pattern
 }
 
-// Parameters for SHOW TABLES commands
+// Parameters for SHOW TABLES and SHOW VIEWS commands
 struct TShowTablesParams {
   // Database to use for SHOW TABLES
   1: optional string db
@@ -254,6 +258,10 @@ struct TShowTablesParams {
   // Optional pattern to match tables names. If not set, all tables from the given
   // database are returned.
   2: optional string show_pattern
+
+  // This specifies the types of tables that should be returned. If not set, all types of
+  // tables are considered when their names are matched against pattern.
+  3: optional set<CatalogService.TImpalaTableType> table_types = []
 }
 
 // Parameters for SHOW FILES commands
@@ -429,6 +437,7 @@ enum TCatalogOpType {
   SHOW_FILES = 13
   SHOW_CREATE_FUNCTION = 14
   DESCRIBE_HISTORY = 15
+  SHOW_VIEWS = 16
 }
 
 // TODO: Combine SHOW requests with a single struct that contains a field
diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index e46a57307..af596629c 100755
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -332,7 +332,7 @@ terminal
   KW_TERMINATED, KW_TEXTFILE, KW_THEN, KW_TIMESTAMP, KW_TINYINT, KW_TRUNCATE, KW_STATS,
   KW_TO, KW_TRUE, KW_UDF, KW_UNBOUNDED, KW_UNCACHED, KW_UNION, KW_UNIQUE, KW_UNKNOWN,
   KW_UNNEST, KW_UNSET, KW_UPDATE, KW_UPDATE_FN, KW_UPSERT, KW_USE, KW_USING, KW_VALIDATE,
-  KW_VALUES, KW_VARCHAR, KW_VIEW, KW_WHEN, KW_WHERE, KW_WITH, KW_ZORDER;
+  KW_VALUES, KW_VARCHAR, KW_VIEW, KW_VIEWS, KW_WHEN, KW_WHERE, KW_WITH, KW_ZORDER;
 
 terminal UNUSED_RESERVED_WORD;
 
@@ -381,6 +381,7 @@ nonterminal UseStmt use_stmt;
 nonterminal SetStmt set_stmt;
 nonterminal SetStmt unset_stmt;
 nonterminal ShowTablesStmt show_tables_stmt;
+nonterminal ShowViewsStmt show_views_stmt;
 nonterminal ShowDbsStmt show_dbs_stmt;
 nonterminal ShowStatsStmt show_stats_stmt, show_partitions_stmt,
   show_range_partitions_stmt, show_hash_schema_stmt;
@@ -680,6 +681,8 @@ stmt ::=
   {: RESULT = use; :}
   | show_tables_stmt:show_tables
   {: RESULT = show_tables; :}
+  | show_views_stmt:show_views
+  {: RESULT = show_views; :}
   | show_dbs_stmt:show_dbs
   {: RESULT = show_dbs; :}
   | show_partitions_stmt:show_partitions
@@ -2929,6 +2932,17 @@ show_tables_stmt ::=
   {: RESULT = new ShowTablesStmt(db, showPattern); :}
   ;
 
+show_views_stmt ::=
+  KW_SHOW KW_VIEWS
+  {: RESULT = new ShowViewsStmt(); :}
+  | KW_SHOW KW_VIEWS show_pattern:showPattern
+  {: RESULT = new ShowViewsStmt(showPattern); :}
+  | KW_SHOW KW_VIEWS KW_IN ident_or_default:db
+  {: RESULT = new ShowViewsStmt(db, null); :}
+  | KW_SHOW KW_VIEWS KW_IN ident_or_default:db show_pattern:showPattern
+  {: RESULT = new ShowViewsStmt(db, showPattern); :}
+  ;
+
 show_dbs_stmt ::=
   KW_SHOW dbs_or_schemas_kw
   {: RESULT = new ShowDbsStmt(); :}
@@ -4617,6 +4631,8 @@ word ::=
   {: RESULT = r.toString(); :}
   | KW_VIEW:r
   {: RESULT = r.toString(); :}
+  | KW_VIEWS:r
+  {: RESULT = r.toString(); :}
   | KW_WHEN:r
   {: RESULT = r.toString(); :}
   | KW_WHERE:r
diff --git a/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java b/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
index c2776d04b..3c02439fc 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
@@ -119,6 +119,7 @@ public class AnalysisContext {
     public boolean isUseStmt() { return stmt_ instanceof UseStmt; }
     public boolean isSetStmt() { return stmt_ instanceof SetStmt; }
     public boolean isShowTablesStmt() { return stmt_ instanceof ShowTablesStmt; }
+    public boolean isShowViewsStmt() { return stmt_ instanceof ShowViewsStmt; }
     public boolean isDescribeHistoryStmt() {
       return stmt_ instanceof DescribeHistoryStmt;
     }
@@ -176,11 +177,11 @@ public class AnalysisContext {
     }
 
     private boolean isViewMetadataStmt() {
-      return isShowFilesStmt() || isShowTablesStmt() || isShowDbsStmt() ||
-          isShowFunctionsStmt() || isShowRolesStmt() || isShowGrantPrincipalStmt() ||
-          isShowCreateTableStmt() || isShowDataSrcsStmt() || isShowStatsStmt() ||
-          isDescribeTableStmt() || isDescribeDbStmt() || isShowCreateFunctionStmt() ||
-          isDescribeHistoryStmt();
+      return isShowFilesStmt() || isShowTablesStmt() || isShowViewsStmt() ||
+          isShowDbsStmt() || isShowFunctionsStmt() || isShowRolesStmt() ||
+          isShowGrantPrincipalStmt() || isShowCreateTableStmt() ||
+          isShowDataSrcsStmt() || isShowStatsStmt() || isDescribeTableStmt() ||
+          isDescribeDbStmt() || isShowCreateFunctionStmt() || isDescribeHistoryStmt();
     }
 
     private boolean isGrantRevokeStmt() {
@@ -210,7 +211,8 @@ public class AnalysisContext {
      */
     public boolean isSingleColumnPrivStmt() {
       return isDescribeTableStmt() || isResetMetadataStmt() || isUseStmt()
-          || isShowTablesStmt() || isAlterTableStmt() || isShowFunctionsStmt();
+          || isShowTablesStmt() || isShowViewsStmt() || isAlterTableStmt()
+          || isShowFunctionsStmt();
     }
 
     public boolean isConvertTableToIcebergStmt() {
@@ -328,6 +330,11 @@ public class AnalysisContext {
       return (ShowTablesStmt) stmt_;
     }
 
+    public ShowViewsStmt getShowViewsStmt() {
+      Preconditions.checkState(isShowViewsStmt());
+      return (ShowViewsStmt) stmt_;
+    }
+
     public ShowDbsStmt getShowDbsStmt() {
       Preconditions.checkState(isShowDbsStmt());
       return (ShowDbsStmt) stmt_;
diff --git a/fe/src/main/java/org/apache/impala/analysis/ShowTablesStmt.java b/fe/src/main/java/org/apache/impala/analysis/ShowTablesOrViewsStmt.java
similarity index 67%
copy from fe/src/main/java/org/apache/impala/analysis/ShowTablesStmt.java
copy to fe/src/main/java/org/apache/impala/analysis/ShowTablesOrViewsStmt.java
index bb2626789..85b7ffc5b 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ShowTablesStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ShowTablesOrViewsStmt.java
@@ -23,22 +23,8 @@ import org.apache.impala.thrift.TShowTablesParams;
 
 import com.google.common.base.Preconditions;
 
-/**
- * Representation of a SHOW TABLES [pattern] statement.
- * Acceptable syntax:
- *
- * SHOW TABLES
- * SHOW TABLES "pattern"
- * SHOW TABLES LIKE "pattern"
- * SHOW TABLES IN database
- * SHOW TABLES IN database "pattern"
- * SHOW TABLES IN database LIKE "pattern"
- *
- * In Hive, the 'LIKE' is optional. Also SHOW TABLES unquotedpattern is accepted
- * by the parser but returns no results. We don't support that syntax.
- */
-public class ShowTablesStmt extends StatementBase {
-  // Pattern to match tables against. | denotes choice, * matches all strings
+public abstract class ShowTablesOrViewsStmt extends StatementBase {
+  // Pattern to match tables or views against. | denotes choice, * matches all strings
   private final String pattern_;
 
   // DB (if any) as seen by the parser
@@ -49,9 +35,11 @@ public class ShowTablesStmt extends StatementBase {
 
   /**
    * Default constructor, which creates a show statement with the default
-   * database and no pattern (which returns all tables in the default database).
+   * database and no pattern which returns either a) all tables and views, or b) all
+   * views in the default database depending on the instantiation (ShowTablesStmt v.s.
+   * ShowViewsStmt).
    */
-  public ShowTablesStmt() {
+  public ShowTablesOrViewsStmt() {
     this(null, null);
   }
 
@@ -59,23 +47,26 @@ public class ShowTablesStmt extends StatementBase {
    * Constructs a show statement against the default database using the supplied
    * pattern.
    */
-  public ShowTablesStmt(String pattern) {
+  public ShowTablesOrViewsStmt(String pattern) {
     this(null, pattern);
   }
 
   /**
    * General purpose constructor which builds a show statement that matches
-   * table names against a given pattern in the supplied database.
+   * either a) all table and view names against a given pattern, or b) all view names in
+   * the supplied database depending on the instantiation (ShowTablesStmt v.s.
+   * ShowViewsStmt).
    *
    * If pattern is null, all tables in the supplied database match.
    * If database is null, the default database is searched.
    */
-  public ShowTablesStmt(String database, String pattern) {
+  public ShowTablesOrViewsStmt(String database, String pattern) {
     this.parsedDb_ = database;
     this.pattern_ = pattern;
     this.postAnalysisDb_ = null;
   }
 
+  public String getParsedDb() { return parsedDb_; }
   public String getPattern() { return pattern_; }
 
   /**
@@ -87,23 +78,6 @@ public class ShowTablesStmt extends StatementBase {
     return postAnalysisDb_;
   }
 
-  @Override
-  public String toSql(ToSqlOptions options) {
-    if (pattern_ == null) {
-      if (parsedDb_ == null) {
-        return "SHOW TABLES";
-      } else {
-        return "SHOW TABLES IN " + parsedDb_;
-      }
-    } else {
-      if (parsedDb_ == null) {
-        return "SHOW TABLES LIKE '" + pattern_ + "'";
-      } else {
-        return "SHOW TABLES IN " + parsedDb_ + " LIKE '" + pattern_ + "'";
-      }
-    }
-  }
-
   @Override
   public void analyze(Analyzer analyzer) throws AnalysisException {
     postAnalysisDb_ = (parsedDb_ == null ? analyzer.getDefaultDb() : parsedDb_);
diff --git a/fe/src/main/java/org/apache/impala/analysis/ShowTablesStmt.java b/fe/src/main/java/org/apache/impala/analysis/ShowTablesStmt.java
index bb2626789..cfb00d0d6 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ShowTablesStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ShowTablesStmt.java
@@ -17,12 +17,6 @@
 
 package org.apache.impala.analysis;
 
-import org.apache.impala.authorization.Privilege;
-import org.apache.impala.common.AnalysisException;
-import org.apache.impala.thrift.TShowTablesParams;
-
-import com.google.common.base.Preconditions;
-
 /**
  * Representation of a SHOW TABLES [pattern] statement.
  * Acceptable syntax:
@@ -37,85 +31,27 @@ import com.google.common.base.Preconditions;
  * In Hive, the 'LIKE' is optional. Also SHOW TABLES unquotedpattern is accepted
  * by the parser but returns no results. We don't support that syntax.
  */
-public class ShowTablesStmt extends StatementBase {
-  // Pattern to match tables against. | denotes choice, * matches all strings
-  private final String pattern_;
-
-  // DB (if any) as seen by the parser
-  private final String parsedDb_;
+public class ShowTablesStmt extends ShowTablesOrViewsStmt {
+  public ShowTablesStmt() { super(null, null); }
 
-  // Set during analysis
-  private String postAnalysisDb_;
-
-  /**
-   * Default constructor, which creates a show statement with the default
-   * database and no pattern (which returns all tables in the default database).
-   */
-  public ShowTablesStmt() {
-    this(null, null);
-  }
-
-  /**
-   * Constructs a show statement against the default database using the supplied
-   * pattern.
-   */
-  public ShowTablesStmt(String pattern) {
-    this(null, pattern);
-  }
-
-  /**
-   * General purpose constructor which builds a show statement that matches
-   * table names against a given pattern in the supplied database.
-   *
-   * If pattern is null, all tables in the supplied database match.
-   * If database is null, the default database is searched.
-   */
-  public ShowTablesStmt(String database, String pattern) {
-    this.parsedDb_ = database;
-    this.pattern_ = pattern;
-    this.postAnalysisDb_ = null;
-  }
+  public ShowTablesStmt(String pattern) { super(null, pattern); }
 
-  public String getPattern() { return pattern_; }
-
-  /**
-   * Can only be called after analysis, returns the name of the database that
-   * this show will search against.
-   */
-  public String getDb() {
-    Preconditions.checkNotNull(postAnalysisDb_);
-    return postAnalysisDb_;
-  }
+  public ShowTablesStmt(String database, String pattern) { super(database, pattern); }
 
   @Override
   public String toSql(ToSqlOptions options) {
-    if (pattern_ == null) {
-      if (parsedDb_ == null) {
+    if (getPattern() == null) {
+      if (getParsedDb() == null) {
         return "SHOW TABLES";
       } else {
-        return "SHOW TABLES IN " + parsedDb_;
+        return "SHOW TABLES IN " + getParsedDb();
       }
     } else {
-      if (parsedDb_ == null) {
-        return "SHOW TABLES LIKE '" + pattern_ + "'";
+      if (getParsedDb() == null) {
+        return "SHOW TABLES LIKE '" + getPattern() + "'";
       } else {
-        return "SHOW TABLES IN " + parsedDb_ + " LIKE '" + pattern_ + "'";
+        return "SHOW TABLES IN " + getParsedDb() + " LIKE '" + getPattern() + "'";
       }
     }
   }
-
-  @Override
-  public void analyze(Analyzer analyzer) throws AnalysisException {
-    postAnalysisDb_ = (parsedDb_ == null ? analyzer.getDefaultDb() : parsedDb_);
-    if (analyzer.getDb(postAnalysisDb_, Privilege.ANY) == null) {
-      throw new AnalysisException(Analyzer.DB_DOES_NOT_EXIST_ERROR_MSG + postAnalysisDb_);
-    }
-  }
-
-  public TShowTablesParams toThrift() {
-    TShowTablesParams params = new TShowTablesParams();
-    params.setShow_pattern(getPattern());
-    params.setDb(getDb());
-    return params;
-  }
 }
diff --git a/fe/src/main/java/org/apache/impala/analysis/ShowViewsStmt.java b/fe/src/main/java/org/apache/impala/analysis/ShowViewsStmt.java
new file mode 100644
index 000000000..0a029e21f
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/ShowViewsStmt.java
@@ -0,0 +1,55 @@
+// 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.impala.analysis;
+
+import org.apache.impala.thrift.TImpalaTableType;
+import org.apache.impala.thrift.TShowTablesParams;
+
+import com.google.common.collect.Sets;
+
+public class ShowViewsStmt extends ShowTablesOrViewsStmt {
+  public ShowViewsStmt() { super(null, null); }
+
+  public ShowViewsStmt(String pattern) { super(null, pattern); }
+
+  public ShowViewsStmt(String database, String pattern) { super(database, pattern); }
+
+  @Override
+  public String toSql(ToSqlOptions options) {
+    if (getPattern() == null) {
+      if (getParsedDb() == null) {
+        return "SHOW VIEWS";
+      } else {
+        return "SHOW VIEWS IN " + getParsedDb();
+      }
+    } else {
+      if (getParsedDb() == null) {
+        return "SHOW VIEWS LIKE '" + getPattern() + "'";
+      } else {
+        return "SHOW VIEWS IN " + getParsedDb() + " LIKE '" + getPattern() + "'";
+      }
+    }
+  }
+
+  @Override
+  public TShowTablesParams toThrift() {
+    TShowTablesParams params = super.toThrift();
+    params.setTable_types(Sets.newHashSet(TImpalaTableType.VIEW));
+    return params;
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/catalog/Catalog.java b/fe/src/main/java/org/apache/impala/catalog/Catalog.java
index a1a1058ae..430f65aef 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Catalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Catalog.java
@@ -44,6 +44,7 @@ import org.apache.impala.compat.MetastoreShim;
 import org.apache.impala.thrift.TCatalogObject;
 import org.apache.impala.thrift.TFunction;
 import org.apache.impala.thrift.THdfsPartition;
+import org.apache.impala.thrift.TImpalaTableType;
 import org.apache.impala.thrift.TPartitionKeyValue;
 import org.apache.impala.thrift.TPrincipalType;
 import org.apache.impala.thrift.TTable;
@@ -233,20 +234,30 @@ public abstract class Catalog implements AutoCloseable {
   }
 
   /**
-   * Returns all tables in 'dbName' that match 'matcher'.
+   * Returns all tables and views in 'dbName' that match 'matcher'.
+   */
+  public List<String> getTableNames(String dbName, PatternMatcher matcher)
+      throws DatabaseNotFoundException {
+    return getTableNames(dbName, matcher, /*tableTypes*/ Collections.emptySet());
+  }
+
+  /**
+   * Returns all tables of types specified in 'tableTypes' under the database 'dbName'
+   * that match 'matcher'.
    *
    * dbName must not be null.
    *
    * Table names are returned unqualified.
    */
-  public List<String> getTableNames(String dbName, PatternMatcher matcher)
+  public List<String> getTableNames(String dbName, PatternMatcher matcher,
+      Set<TImpalaTableType> tableTypes)
       throws DatabaseNotFoundException {
     Preconditions.checkNotNull(dbName);
     Db db = getDb(dbName);
     if (db == null) {
       throw new DatabaseNotFoundException("Database '" + dbName + "' not found");
     }
-    return filterStringsByPattern(db.getAllTableNames(), matcher);
+    return filterStringsByPattern(db.getAllTableNames(tableTypes), matcher);
   }
 
   /**
diff --git a/fe/src/main/java/org/apache/impala/catalog/Db.java b/fe/src/main/java/org/apache/impala/catalog/Db.java
index 6150c83a2..f767ba80a 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Db.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Db.java
@@ -23,8 +23,10 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.impala.analysis.ColumnDef;
@@ -41,6 +43,7 @@ import org.apache.impala.thrift.TFunctionBinaryType;
 import org.apache.impala.thrift.TFunctionCategory;
 import org.apache.impala.thrift.TGetPartialCatalogObjectRequest;
 import org.apache.impala.thrift.TGetPartialCatalogObjectResponse;
+import org.apache.impala.thrift.TImpalaTableType;
 import org.apache.impala.thrift.TPartialDbInfo;
 import org.apache.impala.util.FunctionUtils;
 import org.apache.impala.util.PatternMatcher;
@@ -199,11 +202,23 @@ public class Db extends CatalogObjectImpl implements FeDb {
    */
   public void addTable(Table table) { tableCache_.add(table); }
 
+  @Override
+  public List<String> getAllTableNames() {
+    return getAllTableNames(/*tableTypes*/ Collections.emptySet());
+  }
+
   /**
-   * Gets all table names in the table cache.
+   * Gets all table names in the table cache whose corresponding tables are of a table
+   * type specified in 'tableTypes'. Returns all table names if 'tableTypes' is empty.
    */
   @Override
-  public List<String> getAllTableNames() {
+  public List<String> getAllTableNames(Set<TImpalaTableType> tableTypes) {
+    if (!tableTypes.isEmpty()) {
+      return tableCache_.getValues().stream()
+          .filter(table -> tableTypes.contains(table.getTableType()))
+          .map(table -> table.getName())
+          .collect(Collectors.toList());
+    }
     return Lists.newArrayList(tableCache_.keySet());
   }
 
diff --git a/fe/src/main/java/org/apache/impala/catalog/FeCatalog.java b/fe/src/main/java/org/apache/impala/catalog/FeCatalog.java
index 4d2bee81e..617c9e549 100644
--- a/fe/src/main/java/org/apache/impala/catalog/FeCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/FeCatalog.java
@@ -24,6 +24,7 @@ import org.apache.impala.authorization.AuthorizationPolicy;
 import org.apache.impala.common.InternalException;
 import org.apache.impala.thrift.TCatalogObject;
 import org.apache.impala.thrift.TGetPartitionStatsResponse;
+import org.apache.impala.thrift.TImpalaTableType;
 import org.apache.impala.thrift.TPartitionKeyValue;
 import org.apache.impala.thrift.TUniqueId;
 import org.apache.impala.util.PatternMatcher;
@@ -37,10 +38,13 @@ public interface FeCatalog {
   /** @see Catalog#getDbs(PatternMatcher) */
   List<? extends FeDb> getDbs(PatternMatcher matcher);
 
-  /** @see Catalog#getTableNames(String, PatternMatcher) */
   List<String> getTableNames(String dbName, PatternMatcher matcher)
       throws DatabaseNotFoundException;
 
+  /** @see Catalog#getTableNames(String, PatternMatcher, Set<String>) */
+  List<String> getTableNames(String dbName, PatternMatcher matcher,
+      Set<TImpalaTableType> tableTypes) throws DatabaseNotFoundException;
+
   /** @see Catalog#getTable(String, String) */
   FeTable getTable(String dbName, String tableName)
       throws DatabaseNotFoundException;
diff --git a/fe/src/main/java/org/apache/impala/catalog/FeDb.java b/fe/src/main/java/org/apache/impala/catalog/FeDb.java
index f8cc4715d..468ff0dc8 100644
--- a/fe/src/main/java/org/apache/impala/catalog/FeDb.java
+++ b/fe/src/main/java/org/apache/impala/catalog/FeDb.java
@@ -17,6 +17,7 @@
 package org.apache.impala.catalog;
 
 import java.util.Comparator;
+import java.util.Set;
 import java.util.List;
 
 import org.apache.hadoop.hive.metastore.api.Database;
@@ -27,6 +28,7 @@ import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.ImpalaRuntimeException;
 import org.apache.impala.thrift.TDatabase;
 import org.apache.impala.thrift.TFunctionCategory;
+import org.apache.impala.thrift.TImpalaTableType;
 import org.apache.impala.util.PatternMatcher;
 
 /**
@@ -70,10 +72,16 @@ public interface FeDb extends HasName {
   FeTable getTableIfCached(String tbl);
 
   /**
-   * @return the names of the tables within this database
+   * @return the names of the tables and views within this database
    */
   List<String> getAllTableNames();
 
+  /**
+   * Gets all table names in the table cache whose corresponding tables are of a table
+   * type specified in 'tableTypes'. Returns all table names if 'tableTypes' is empty.
+   */
+  List<String> getAllTableNames(Set<TImpalaTableType> tableTypes);
+
   /**
    * @return true if this is a system database (i.e. cannot be dropped,
    * modified, etc)
diff --git a/fe/src/main/java/org/apache/impala/catalog/Hive3MetastoreShimBase.java b/fe/src/main/java/org/apache/impala/catalog/Hive3MetastoreShimBase.java
index feb374560..4f4bca324 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Hive3MetastoreShimBase.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Hive3MetastoreShimBase.java
@@ -266,7 +266,7 @@ public class Hive3MetastoreShimBase {
           .put("MANAGED_TABLE", TImpalaTableType.TABLE)
           .put("INDEX_TABLE", TImpalaTableType.TABLE)
           .put("VIRTUAL_VIEW", TImpalaTableType.VIEW)
-          .put("MATERIALIZED_VIEW", TImpalaTableType.VIEW).build();
+          .put("MATERIALIZED_VIEW", TImpalaTableType.MATERIALIZED_VIEW).build();
 
   /**
    * Method which maps Metastore's TableType to Impala's table type. In metastore 2
@@ -287,8 +287,9 @@ public class Hive3MetastoreShimBase {
       case MANAGED_TABLE:
         return TImpalaTableType.TABLE;
       case VIRTUAL_VIEW:
-      case MATERIALIZED_VIEW:
         return TImpalaTableType.VIEW;
+      case MATERIALIZED_VIEW:
+        return TImpalaTableType.MATERIALIZED_VIEW;
       default:
         return defaultTableType;
     }
diff --git a/fe/src/main/java/org/apache/impala/catalog/IncompleteTable.java b/fe/src/main/java/org/apache/impala/catalog/IncompleteTable.java
index e80c6337e..04a962fc3 100644
--- a/fe/src/main/java/org/apache/impala/catalog/IncompleteTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/IncompleteTable.java
@@ -122,6 +122,8 @@ public class IncompleteTable extends Table implements FeIncompleteTable {
     }
     if (tableType_ == TImpalaTableType.VIEW) {
       table.setTable_type(TTableType.VIEW);
+    } else if (tableType_ == TImpalaTableType.MATERIALIZED_VIEW) {
+      table.setTable_type(TTableType.MATERIALIZED_VIEW);
     } else {
       table.setTable_type(TTableType.UNLOADED_TABLE);
     }
diff --git a/fe/src/main/java/org/apache/impala/catalog/Table.java b/fe/src/main/java/org/apache/impala/catalog/Table.java
index 5af181169..b42874beb 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Table.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Table.java
@@ -494,10 +494,12 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
     CatalogInterners.internFieldsInPlace(msTbl);
     Table table = null;
     // Create a table of appropriate type
-    if (TImpalaTableType.VIEW ==
-        MetastoreShim.mapToInternalTableType(msTbl.getTableType())) {
-      if (msTbl.getTableType().equalsIgnoreCase("MATERIALIZED_VIEW") &&
-          HdfsFileFormat.isHdfsInputFormatClass(msTbl.getSd().getInputFormat())) {
+    TImpalaTableType tableType =
+        MetastoreShim.mapToInternalTableType(msTbl.getTableType());
+    if (tableType == TImpalaTableType.VIEW) {
+        table = new View(msTbl, db, msTbl.getTableName(), msTbl.getOwner());
+    } else if (tableType == TImpalaTableType.MATERIALIZED_VIEW) {
+      if (HdfsFileFormat.isHdfsInputFormatClass(msTbl.getSd().getInputFormat())) {
         table = new MaterializedViewHdfsTable(msTbl, db, msTbl.getTableName(),
             msTbl.getOwner());
       } else {
@@ -535,6 +537,8 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
       TImpalaTableType tblType;
       if (thriftTable.getTable_type() == TTableType.VIEW) {
         tblType = TImpalaTableType.VIEW;
+      } else if (thriftTable.getTable_type() == TTableType.MATERIALIZED_VIEW) {
+        tblType = TImpalaTableType.MATERIALIZED_VIEW;
       } else {
         // If the table is unloaded or --pull_table_types_and_comments flag is not set,
         // keep the legacy behavior as showing the table type as TABLE.
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalCatalog.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalCatalog.java
index 26f86823e..e6c484c08 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/LocalCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalCatalog.java
@@ -17,6 +17,7 @@
 
 package org.apache.impala.catalog.local;
 
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -47,6 +48,7 @@ import org.apache.impala.thrift.TCatalogObject;
 import org.apache.impala.thrift.TCatalogObjectType;
 import org.apache.impala.thrift.TDataSource;
 import org.apache.impala.thrift.TGetPartitionStatsResponse;
+import org.apache.impala.thrift.TImpalaTableType;
 import org.apache.impala.thrift.TPartitionKeyValue;
 import org.apache.impala.thrift.TUniqueId;
 import org.apache.impala.util.PatternMatcher;
@@ -119,7 +121,16 @@ public class LocalCatalog implements FeCatalog {
   @Override
   public List<String> getTableNames(String dbName, PatternMatcher matcher)
       throws DatabaseNotFoundException {
-    return Catalog.filterStringsByPattern(getDbOrThrow(dbName).getAllTableNames(), matcher);
+    return getTableNames(dbName, matcher, /*tableTypes*/ Collections.emptySet());
+  }
+
+  @Override
+  public List<String> getTableNames(String dbName, PatternMatcher matcher,
+      Set<TImpalaTableType> tableTypes)
+      throws DatabaseNotFoundException {
+    FeDb db = getDbOrThrow(dbName);
+    return Catalog.filterStringsByPattern(db.getAllTableNames(tableTypes),
+        matcher);
   }
 
   @Override
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalDb.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalDb.java
index 82482be3c..7c50b414b 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/LocalDb.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalDb.java
@@ -19,9 +19,11 @@ package org.apache.impala.catalog.local;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.stream.Collectors;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Table;
@@ -40,6 +42,7 @@ import org.apache.impala.common.ImpalaRuntimeException;
 import org.apache.impala.thrift.TBriefTableMeta;
 import org.apache.impala.thrift.TDatabase;
 import org.apache.impala.thrift.TFunctionCategory;
+import org.apache.impala.thrift.TImpalaTableType;
 import org.apache.impala.util.FunctionUtils;
 import org.apache.impala.util.PatternMatcher;
 import org.apache.thrift.TException;
@@ -155,7 +158,19 @@ public class LocalDb implements FeDb {
 
   @Override
   public List<String> getAllTableNames() {
+    return getAllTableNames(/*tableTypes*/ Collections.emptySet());
+  }
+
+  @Override
+  public List<String> getAllTableNames(Set<TImpalaTableType> tableTypes) {
     loadTableNames();
+    if (!tableTypes.isEmpty()) {
+      return tables_.values().stream()
+          .filter(table ->
+              tableTypes.stream().anyMatch(type -> type.equals(table.getTableType())))
+          .map(table -> table.getName())
+          .collect(Collectors.toList());
+    }
     return ImmutableList.copyOf(tables_.keySet());
   }
 
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index 8e77a0bb3..88e975497 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -168,6 +168,7 @@ import org.apache.impala.thrift.TDdlType;
 import org.apache.impala.thrift.TDescribeHistoryParams;
 import org.apache.impala.thrift.TDescribeOutputStyle;
 import org.apache.impala.thrift.TDescribeResult;
+import org.apache.impala.thrift.TImpalaTableType;
 import org.apache.impala.thrift.TIcebergDmlFinalizeParams;
 import org.apache.impala.thrift.TIcebergOperation;
 import org.apache.impala.thrift.TExecRequest;
@@ -584,6 +585,11 @@ public class Frontend {
       ddl.setShow_tables_params(analysis.getShowTablesStmt().toThrift());
       metadata.setColumns(Arrays.asList(
           new TColumn("name", Type.STRING.toThrift())));
+    } else if (analysis.isShowViewsStmt()) {
+      ddl.op_type = TCatalogOpType.SHOW_VIEWS;
+      ddl.setShow_tables_params(analysis.getShowViewsStmt().toThrift());
+      metadata.setColumns(Arrays.asList(
+          new TColumn("name", Type.STRING.toThrift())));
     } else if (analysis.isShowDbsStmt()) {
       ddl.op_type = TCatalogOpType.SHOW_DBS;
       ddl.setShow_dbs_params(analysis.getShowDbsStmt().toThrift());
@@ -1130,17 +1136,22 @@ public class Frontend {
     }
   }
 
+  public List<String> getTableNames(String dbName, PatternMatcher matcher, User user)
+      throws ImpalaException {
+    return getTableNames(dbName, matcher, user, /*tableTypes*/ Collections.emptySet());
+  }
+
   /**
-   * Returns all tables in database 'dbName' that match the pattern of 'matcher' and are
-   * accessible to 'user'.
+   * Returns tables of types specified in 'tableTypes' in database 'dbName' that
+   * match the pattern of 'matcher' and are accessible to 'user'.
    */
   public List<String> getTableNames(String dbName, PatternMatcher matcher,
-      User user) throws ImpalaException {
+      User user, Set<TImpalaTableType> tableTypes) throws ImpalaException {
     RetryTracker retries = new RetryTracker(
         String.format("fetching %s table names", dbName));
     while (true) {
       try {
-        return doGetTableNames(dbName, matcher, user);
+        return doGetTableNames(dbName, matcher, user, tableTypes);
       } catch(InconsistentMetadataFetchException e) {
         retries.handleRetryOrThrow(e);
       }
@@ -1176,9 +1187,10 @@ public class Frontend {
   }
 
   private List<String> doGetTableNames(String dbName, PatternMatcher matcher,
-      User user) throws ImpalaException {
+      User user, Set<TImpalaTableType> tableTypes)
+      throws ImpalaException {
     FeCatalog catalog = getCatalog();
-    List<String> tblNames = catalog.getTableNames(dbName, matcher);
+    List<String> tblNames = catalog.getTableNames(dbName, matcher, tableTypes);
 
     boolean needsAuthChecks = authzFactory_.getAuthorizationConfig().isEnabled()
                               && !userHasAccessForWholeDb(user, dbName);
diff --git a/fe/src/main/java/org/apache/impala/service/JniFrontend.java b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
index 763256a0c..1f2213e15 100644
--- a/fe/src/main/java/org/apache/impala/service/JniFrontend.java
+++ b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
@@ -278,7 +278,8 @@ public class JniFrontend {
 
     Preconditions.checkState(!params.isSetSession() || user != null );
     List<String> tables = frontend_.getTableNames(params.db,
-        PatternMatcher.createHivePatternMatcher(params.pattern), user);
+        PatternMatcher.createHivePatternMatcher(params.pattern), user,
+        params.getTable_types());
 
     TGetTablesResult result = new TGetTablesResult();
     result.setTables(tables);
diff --git a/fe/src/main/jflex/sql-scanner.flex b/fe/src/main/jflex/sql-scanner.flex
index 481a7bd2f..56598c1fa 100644
--- a/fe/src/main/jflex/sql-scanner.flex
+++ b/fe/src/main/jflex/sql-scanner.flex
@@ -291,6 +291,7 @@ import org.apache.impala.thrift.TReservedWordsVersion;
     keywordMap.put("values", SqlParserSymbols.KW_VALUES);
     keywordMap.put("varchar", SqlParserSymbols.KW_VARCHAR);
     keywordMap.put("view", SqlParserSymbols.KW_VIEW);
+    keywordMap.put("views", SqlParserSymbols.KW_VIEWS);
     keywordMap.put("when", SqlParserSymbols.KW_WHEN);
     keywordMap.put("where", SqlParserSymbols.KW_WHERE);
     keywordMap.put("with", SqlParserSymbols.KW_WITH);
diff --git a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
index d8972bc7a..3bd6fda9e 100755
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -1975,10 +1975,17 @@ public class ParserTest extends FrontendTestBase {
   public void TestShow() {
     // Short form ok
     ParsesOk("SHOW TABLES");
+    ParsesOk("SHOW VIEWS");
     // Well-formed pattern
     ParsesOk("SHOW TABLES 'tablename|othername'");
+    ParsesOk("SHOW VIEWS 'tablename|othername'");
+    ParsesOk("SHOW TABLES LIKE 'tablename|othername'");
+    ParsesOk("SHOW VIEWS LIKE 'tablename|othername'");
+    ParsesOk("SHOW TABLES IN db LIKE 'tablename|othername'");
+    ParsesOk("SHOW VIEWS IN db LIKE 'tablename|othername'");
     // Empty pattern ok
     ParsesOk("SHOW TABLES ''");
+    ParsesOk("SHOW VIEWS ''");
     // Databases
     ParsesOk("SHOW DATABASES");
     ParsesOk("SHOW SCHEMAS");
@@ -2028,6 +2035,7 @@ public class ParserTest extends FrontendTestBase {
     ParserError("SHOW");
     // Malformed pattern (no quotes)
     ParserError("SHOW TABLES tablename");
+    ParserError("SHOW VIEWS tablename");
     // Invalid SHOW DATA SOURCE statements
     ParserError("SHOW DATA");
     ParserError("SHOW SOURCE");
diff --git a/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java b/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
index c10d4b9de..1036e1c2b 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
@@ -1895,4 +1895,28 @@ public class ToSqlTest extends FrontendTestBase {
     testToSql(":shutdown('hostname', 1000)");
     testToSql(":shutdown(1000)");
   }
+
+  /**
+   * Test SHOW TABLES statements are output correctly.
+   */
+  @Test
+  public void testShowTables() {
+    testToSql("SHOW TABLES", "default", "SHOW TABLES");
+    testToSql("SHOW TABLES IN functional");
+    testToSql("SHOW TABLES LIKE 'alltypes*'", "functional",
+        "SHOW TABLES LIKE 'alltypes*'");
+    testToSql("SHOW TABLES IN functional LIKE 'alltypes*'");
+  }
+
+  /**
+   * Test SHOW VIEWS statements are output correctly.
+   */
+  @Test
+  public void testShowViews() {
+    testToSql("SHOW VIEWS", "default", "SHOW VIEWS");
+    testToSql("SHOW VIEWS IN functional");
+    testToSql("SHOW VIEWS LIKE 'alltypes*'", "functional",
+        "SHOW VIEWS LIKE 'alltypes*'");
+    testToSql("SHOW VIEWS IN functional LIKE 'alltypes*'");
+  }
 }
diff --git a/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java b/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
index 68b4f0db1..5e224f318 100644
--- a/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
+++ b/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
@@ -1253,6 +1253,16 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
     }
     test.error(accessError("functional.*.*"));
 
+    // Show views.
+    test = authorize("show views in functional");
+    // We exclude TPrivilegeLevel.RWSTORAGE because of the same reason mentioned above.
+    for (TPrivilegeLevel privilege: allExcept(TPrivilegeLevel.RWSTORAGE)) {
+      test.ok(onServer(privilege))
+          .ok(onDatabase("functional", privilege))
+          .ok(onTable("functional", "alltypes_views", privilege));
+    }
+    test.error(accessError("functional.*.*"));
+
     // Show functions.
     test = authorize("show functions in functional");
     for (TPrivilegeLevel privilege: viewMetadataPrivileges()) {
@@ -1264,9 +1274,15 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
     // Show tables in system database should always be allowed.
     authorize("show tables in _impala_builtins").ok();
 
+    // Show views in system database should always be allowed.
+    authorize("show views in _impala_builtins").ok();
+
     // Show tables for non-existent database.
     authorize("show tables in nodb").error(accessError("nodb"));
 
+    // Show views for non-existent database.
+    authorize("show views in nodb").error(accessError("nodb"));
+
     // Show partitions, table stats, and column stats
     for (AuthzTest authzTest: new AuthzTest[]{
         authorize("show partitions functional.alltypes"),
diff --git a/testdata/workloads/functional-query/queries/QueryTest/show_views.test b/testdata/workloads/functional-query/queries/QueryTest/show_views.test
new file mode 100644
index 000000000..737c79f5a
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/show_views.test
@@ -0,0 +1,176 @@
+====
+---- QUERY
+# use functional
+use functional
+====
+---- QUERY
+# Simple pattern literal
+show views "alltypes_view"
+---- RESULTS
+'alltypes_view'
+---- TYPES
+STRING
+====
+---- QUERY
+# Pattern with wildcard
+show views "all*types*"
+---- RESULTS
+'alltypes_dp_2_view_1'
+'alltypes_dp_2_view_2'
+'alltypes_hive_view'
+'alltypes_parens'
+'alltypes_view'
+'alltypes_view_sub'
+---- TYPES
+STRING
+====
+---- QUERY
+# Empty string matches nothing
+show views ""
+---- RESULTS
+---- TYPES
+STRING
+====
+---- QUERY
+# Check that regex metacharacters are being escaped
+show views '+alltypes_view'
+---- RESULTS
+---- TYPES
+STRING
+====
+---- QUERY
+# Fully qualified names will not match
+show views "functional.alltypes_view"
+---- RESULTS
+---- TYPES
+STRING
+====
+---- QUERY
+# We have no SHOW VIES with no pattern test since the set of views is
+# changing often, and may be different depending on whether benchmark data is
+# loaded, for example.
+show views '*'
+---- RESULTS: VERIFY_IS_SUBSET
+'alltypes_dp_2_view_1'
+'alltypes_dp_2_view_2'
+'alltypes_hive_view'
+'alltypes_parens'
+'alltypes_view'
+'alltypes_view_sub'
+'complex_view'
+'hive_view'
+'view_view'
+---- TYPES
+STRING
+====
+---- QUERY
+# Choice amongst patterns
+show views "alltypes_view|complex_view|hive_view"
+---- RESULTS
+'alltypes_view'
+'complex_view'
+'hive_view'
+---- TYPES
+STRING
+====
+---- QUERY
+# Show views in
+show views in functional
+---- RESULTS: VERIFY_IS_SUBSET
+'alltypes_dp_2_view_1'
+'alltypes_dp_2_view_2'
+'alltypes_hive_view'
+'alltypes_parens'
+'alltypes_view'
+'alltypes_view_sub'
+'complex_view'
+'hive_view'
+'view_view'
+---- TYPES
+STRING
+====
+---- HIVE_QUERY
+# This test case verifies that HiveServer2 treats 'materialized_view' as a table instead
+# of a view.
+show views in functional
+---- RESULTS: VERIFY_IS_SUBSET
+'alltypes_dp_2_view_1'
+'alltypes_dp_2_view_2'
+'alltypes_hive_view'
+'alltypes_parens'
+'alltypes_view'
+'alltypes_view_sub'
+'complex_view'
+'hive_view'
+'view_view'
+---- TYPES
+STRING
+====
+---- QUERY
+# Show views in with a pattern
+show views in functional like 'alltypes*'
+---- RESULTS
+'alltypes_dp_2_view_1'
+'alltypes_dp_2_view_2'
+'alltypes_hive_view'
+'alltypes_parens'
+'alltypes_view'
+'alltypes_view_sub'
+---- TYPES
+STRING
+====
+---- QUERY
+# Impala only considers '*' and '|' as meta-characters in SHOW statements
+# See IMPALA-3744
+show views in functional like 'alltypes.'
+---- RESULTS
+---- TYPES
+STRING
+====
+---- QUERY
+show views in functional like 'alltypes.*'
+---- RESULTS
+---- TYPES
+STRING
+====
+---- QUERY
+show views in functional like 'alltypes%'
+---- RESULTS
+---- TYPES
+STRING
+====
+---- QUERY
+show views in functional like 'alltypes_'
+---- RESULTS
+---- TYPES
+STRING
+====
+---- QUERY
+# Coverage of syntax variations.
+show views in functional 'alltypes_view'
+---- RESULTS
+'alltypes_view'
+---- TYPES
+STRING
+====
+---- QUERY
+show views in functional like 'alltypes_view'
+---- RESULTS
+'alltypes_view'
+---- TYPES
+STRING
+====
+---- QUERY
+show views 'alltypes_view'
+---- RESULTS
+'alltypes_view'
+---- TYPES
+STRING
+====
+---- QUERY
+show views like 'alltypes_view'
+---- RESULTS
+'alltypes_view'
+---- TYPES
+STRING
+====
diff --git a/tests/custom_cluster/test_preload_table_types.py b/tests/custom_cluster/test_preload_table_types.py
index bf57200de..0491d341c 100644
--- a/tests/custom_cluster/test_preload_table_types.py
+++ b/tests/custom_cluster/test_preload_table_types.py
@@ -42,7 +42,8 @@ class TestPreLoadTableTypes(CustomClusterTestSuite):
     assert types_and_comments['functional.alltypestiny'] == ('TABLE', 'Tiny table')
     assert types_and_comments['functional.alltypes_view'] == ('VIEW', 'View on alltypes')
     assert types_and_comments['functional.alltypes'] == ('TABLE', '')
-    assert types_and_comments['functional_orc_def.materialized_view'] == ('VIEW', '')
+    assert types_and_comments['functional_orc_def.materialized_view'] == \
+        ('MATERIALIZED_VIEW', '')
 
   def verify_table_types_and_comments(self, unique_database):
     n = self.get_impalad_cluster_size()
diff --git a/tests/custom_cluster/test_show_views_statements.py b/tests/custom_cluster/test_show_views_statements.py
new file mode 100644
index 000000000..0e4896fa9
--- /dev/null
+++ b/tests/custom_cluster/test_show_views_statements.py
@@ -0,0 +1,44 @@
+# 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.
+
+# Impala tests for queries that query metadata and set session settings
+
+from __future__ import absolute_import
+
+from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
+
+LOCAL_CATALOG_IMPALAD_ARGS = "--use_local_catalog=true"
+LOCAL_CATALOG_CATALOGD_ARGS = "--catalog_topic_mode=minimal"
+PULL_TABLE_TYPES_FLAG = "--pull_table_types_and_comments=true"
+
+
+class TestShowViewsStatements(CustomClusterTestSuite):
+
+  @classmethod
+  def get_workload(self):
+    return 'functional-query'
+
+  @CustomClusterTestSuite.with_args(
+    catalogd_args=PULL_TABLE_TYPES_FLAG)
+  def test_show_views(self, vector):
+    self.run_test_case('QueryTest/show_views', vector)
+
+  @CustomClusterTestSuite.with_args(
+    impalad_args=LOCAL_CATALOG_IMPALAD_ARGS,
+    catalogd_args="{0} {1}".format(LOCAL_CATALOG_CATALOGD_ARGS, PULL_TABLE_TYPES_FLAG))
+  def test_show_views_local_catalog(self, vector):
+    self.run_test_case('QueryTest/show_views', vector)