You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by gj...@apache.org on 2020/05/07 22:29:25 UTC

[phoenix] branch master updated: PHOENIX-5543: Implement SHOW TABLES/SCHEMAS sql commands

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 116b9bd  PHOENIX-5543: Implement SHOW TABLES/SCHEMAS sql commands
116b9bd is described below

commit 116b9bd667b0cf0e22a45c42e878e97c20c1fc58
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Tue Oct 22 14:14:10 2019 -0700

    PHOENIX-5543: Implement SHOW TABLES/SCHEMAS sql commands
    
    This patch adds new SQL grammar like following
    
    - SHOW SCHEMAS [like '<pattern>']
    - SHOW TABLES [IN <schema>] [like '<pattern']
    
    Example invocations:
    
    - show schemas
    - show scemas like 'SYS%'
    - show tables
    - show tables in SYSTEM
    - show tables in SYSTEM like 'CAT%'
    
    The current way of fetching this information is by using
    !tables and !schemas via sqlline JDBC support but that is
    not flexible enough for the end users to add more fitlers.
    This approach is more inline with what other databases do.
    
    Added test coverage in parser tests and core e2e tests.
---
 .../phoenix/end2end/QueryDatabaseMetaDataIT.java   |  58 ++++
 .../phoenix/end2end/TenantSpecificTablesDDLIT.java |  44 ++-
 phoenix-core/src/main/antlr3/PhoenixSQL.g          |  14 +
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java      | 280 ++---------------
 .../org/apache/phoenix/jdbc/PhoenixStatement.java  |  42 +++
 .../org/apache/phoenix/parse/ParseNodeFactory.java |   7 +
 .../apache/phoenix/parse/ShowSchemasStatement.java |  70 +++++
 .../org/apache/phoenix/parse/ShowStatement.java    |  38 +++
 .../apache/phoenix/parse/ShowTablesStatement.java  |  92 ++++++
 .../java/org/apache/phoenix/util/QueryUtil.java    | 331 +++++++++++++++++++++
 .../org/apache/phoenix/parse/QueryParserTest.java  |  24 +-
 .../org/apache/phoenix/pherf/util/PhoenixUtil.java |   5 -
 12 files changed, 730 insertions(+), 275 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
index 83f7c1b..f1ff66e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
@@ -25,6 +25,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_MUTEX_TABLE
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE;
 import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
 import static org.apache.phoenix.util.TestUtil.CUSTOM_ENTITY_DATA_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.ENTITY_HISTORY_TABLE_NAME;
 import static org.apache.phoenix.util.TestUtil.PTSDB_NAME;
 import static org.apache.phoenix.util.TestUtil.STABLE_NAME;
 import static org.apache.phoenix.util.TestUtil.TABLE_WITH_SALTING;
@@ -44,7 +45,9 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.HashSet;
 import java.util.Properties;
+import java.util.Set;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
@@ -52,6 +55,7 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
@@ -356,6 +360,60 @@ public class QueryDatabaseMetaDataIT extends ParallelStatsDisabledIT {
     }
 
     @Test
+    public void testShowSchemas() throws SQLException {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            ResultSet rs = conn.prepareStatement("show schemas").executeQuery();
+            assertTrue(rs.next());
+            assertEquals("SYSTEM", rs.getString("TABLE_SCHEM"));
+            assertEquals(null, rs.getString("TABLE_CATALOG"));
+            assertFalse(rs.next());
+            // Create another schema and make sure it is listed.
+            String schema = "showschemastest_" + generateUniqueName();
+            String fullTable = schema + "." + generateUniqueName();
+            ensureTableCreated(getUrl(), fullTable, ENTITY_HISTORY_TABLE_NAME, null);
+            // show schemas
+            rs = conn.prepareStatement("show schemas").executeQuery();
+            Set<String> schemas = new HashSet<>();
+            while (rs.next()) {
+                schemas.add(rs.getString("TABLE_SCHEM"));
+                assertEquals(null, rs.getString("TABLE_CATALOG"));
+            }
+            assertEquals(2, schemas.size());
+            assertTrue(schemas.contains("SYSTEM"));
+            assertTrue(schemas.contains(schema.toUpperCase()));
+            // show schemas like 'SYST%' and only SYSTEM should show up.
+            rs = conn.prepareStatement("show schemas like 'SYST%'").executeQuery();
+            assertTrue(rs.next());
+            assertEquals("SYSTEM", rs.getString("TABLE_SCHEM"));
+            assertEquals(null, rs.getString("TABLE_CATALOG"));
+            assertFalse(rs.next());
+        }
+    }
+
+    @Test
+    public void testShowTables() throws SQLException {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            // List all the tables in a particular schema.
+            ResultSet rs = conn.prepareStatement("show tables in SYSTEM").executeQuery();
+            Set<String> tables = new HashSet<>();
+            while (rs.next()) {
+                tables.add(rs.getString("TABLE_NAME"));
+                assertEquals("SYSTEM", rs.getString("TABLE_SCHEM"));
+            }
+            assertEquals(8, tables.size());
+            assertTrue(tables.contains("CATALOG"));
+            assertTrue(tables.contains("FUNCTION"));
+
+            tables.clear();
+            // Add a filter on the table name.
+            rs = conn.prepareStatement("show tables in SYSTEM like 'FUNC%'").executeQuery();
+            while (rs.next()) tables.add(rs.getString("TABLE_NAME"));
+            assertEquals(1, tables.size());
+            assertTrue(tables.contains("FUNCTION"));
+        }
+    }
+
+    @Test
     public void testSchemaMetadataScan() throws SQLException {
         String table1 = generateUniqueName();
         String schema1 = "Z_" + generateUniqueName();
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
index cc5c92d..b6979ab 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
@@ -35,6 +35,7 @@ import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
+import java.sql.Driver;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
@@ -430,14 +431,51 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
 			//Expected
 		}
 	}
-    
+
+	@Test
+    public void testShowTablesMultiTenant() throws Exception {
+        // Each tenant should only be able list tables corresponding to their TENANT_ID
+        String tenantId2 = "T_" + generateUniqueName();
+        String secondTenantConnectionURL =
+            PHOENIX_JDBC_TENANT_SPECIFIC_URL.replace(TENANT_ID,  tenantId2);
+        String tenantTable2 = "V_" + generateUniqueName();
+        createTestTable(
+            secondTenantConnectionURL, TENANT_TABLE_DDL.replace(TENANT_TABLE_NAME, tenantTable2));
+
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        // Non-tenant connections should list all the tables.
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            Set<String> tables = new HashSet<>();
+            ResultSet rs = conn.prepareStatement("show tables").executeQuery();
+            while (rs.next()) {
+              tables.add(rs.getString("TABLE_NAME"));
+            }
+            assertTrue(tables.contains(PARENT_TABLE_NAME));
+            assertTrue(tables.contains(TENANT_TABLE_NAME));
+            assertTrue(tables.contains(tenantTable2));
+        }
+        // Tenant specific connections should not list tables from other tenants.
+        try (Connection conn = DriverManager.getConnection(secondTenantConnectionURL, props)) {
+            Set<String> tables = new HashSet<>();
+            ResultSet rs = conn.prepareStatement("show tables").executeQuery();
+            while (rs.next()) {
+              tables.add(rs.getString("TABLE_NAME"));
+            }
+            assertTrue(tables.contains(PARENT_TABLE_NAME));
+            assertFalse(tables.contains(TENANT_TABLE_NAME));
+            assertTrue(tables.contains(tenantTable2));
+        }
+    }
+
     @Test
     public void testTableMetadataScan() throws Exception {
         // create a tenant table with same name for a different tenant to make sure we are not picking it up in metadata scans for TENANT_ID
         String tenantId2 = "T_" + generateUniqueName();
-        String secondTenatConnectionURL = PHOENIX_JDBC_TENANT_SPECIFIC_URL.replace(TENANT_ID,  tenantId2);
+        String secondTenantConnectionURL =
+            PHOENIX_JDBC_TENANT_SPECIFIC_URL.replace(TENANT_ID,  tenantId2);
         String tenantTable2 = "V_" + generateUniqueName();
-        createTestTable(secondTenatConnectionURL, TENANT_TABLE_DDL.replace(TENANT_TABLE_NAME, tenantTable2));
+        createTestTable(
+            secondTenantConnectionURL, TENANT_TABLE_DDL.replace(TENANT_TABLE_NAME, tenantTable2));
         
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index ac8bfb1..1763b1a 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -73,6 +73,7 @@ tokens
     SESSION='session';
     TABLE='table';
     SCHEMA='schema';
+    SCHEMAS='schemas';
     ADD='add';
     SPLIT='split';
     EXPLAIN='explain';
@@ -147,6 +148,7 @@ tokens
     IMMUTABLE = 'immutable';
     GRANT = 'grant';
     REVOKE = 'revoke';
+    SHOW = 'show';
 }
 
 
@@ -425,6 +427,7 @@ oneStatement returns [BindableStatement ret]
     |   s=drop_index_node
     |   s=alter_index_node
     |   s=alter_table_node
+    |   s=show_node
     |   s=trace_node
     |   s=create_function_node
     |   s=drop_function_node
@@ -487,6 +490,12 @@ revoke_permission_node returns [ChangePermsStatement ret]
         }
     ;
 
+// Parse a show statement. SHOW TABLES, SHOW SCHEMAS ...
+show_node returns [ShowStatement ret]
+    :   SHOW TABLES (IN schema=identifier)? (LIKE pattern=string_literal)? { $ret = factory.showTablesStatement(schema, pattern); }
+    |   SHOW SCHEMAS (LIKE pattern=string_literal)? { $ret = factory.showSchemasStatement(pattern); }
+    ;
+
 // Parse a create view statement.
 create_view_node returns [CreateTableStatement ret]
     :   CREATE VIEW (IF NOT ex=EXISTS)? t=from_table_name 
@@ -526,6 +535,11 @@ int_literal_or_bind returns [ParseNode ret]
     | b=bind_expression { $ret = b; }
     ;
 
+// Returns the normalized string literal
+string_literal returns [String ret]
+    :   s=STRING_LITERAL { ret = SchemaUtil.normalizeLiteral(factory.literal(s.getText())); }
+    ;
+
 // Parse a drop sequence statement.
 drop_sequence_node returns [DropSequenceStatement ret]
     :   DROP SEQUENCE  (IF ex=EXISTS)? t=from_table_name
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index 72d0701..c65026f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -23,12 +23,10 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.RowIdLifetime;
 import java.sql.SQLException;
-import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
@@ -51,26 +49,16 @@ import org.apache.phoenix.expression.LikeExpression;
 import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
 import org.apache.phoenix.expression.StringBasedLikeExpression;
-import org.apache.phoenix.expression.function.ExternalSqlTypeIdFunction;
-import org.apache.phoenix.expression.function.IndexStateNameFunction;
-import org.apache.phoenix.expression.function.SQLIndexTypeFunction;
-import org.apache.phoenix.expression.function.SQLTableTypeFunction;
-import org.apache.phoenix.expression.function.SQLViewTypeFunction;
-import org.apache.phoenix.expression.function.SqlTypeNameFunction;
-import org.apache.phoenix.expression.function.TransactionProviderNameFunction;
 import org.apache.phoenix.hbase.index.util.VersionUtil;
 import org.apache.phoenix.iterate.MaterializedResultIterator;
 import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.parse.LikeParseNode.LikeType;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnImpl;
 import org.apache.phoenix.schema.PDatum;
-import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.RowKeyValueAccessor;
 import org.apache.phoenix.schema.SortOrder;
@@ -86,12 +74,12 @@ import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.PhoenixKeyValueUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
 
 import com.google.common.collect.Lists;
 
-
 /**
  *
  * JDBC DatabaseMetaData implementation of Phoenix.
@@ -485,17 +473,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
 
     @Override
     public ResultSet getCatalogs() throws SQLException {
-        List<String> parameterValues = new ArrayList<String>(4);
-        StringBuilder buf = new StringBuilder("select \n" +
-                " DISTINCT " + TENANT_ID + " " + TABLE_CAT +
-                " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS +
-                " where " + COLUMN_NAME + " is null" +
-                " and " + COLUMN_FAMILY + " is null" +
-                " and " + TENANT_ID + " is not null");
-        addTenantIdFilter(buf, null, parameterValues);
-        buf.append(" order by " + TENANT_ID);
-        PreparedStatement stmt = connection.prepareStatement(buf.toString());
-        setParameters(stmt, parameterValues);
+        PreparedStatement stmt = QueryUtil.getCatalogsStmt(connection);
         return stmt.executeQuery();
     }
 
@@ -512,29 +490,6 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
 
     public static final String GLOBAL_TENANANTS_ONLY = "null";
 
-    private void addTenantIdFilter(StringBuilder buf, String tenantIdPattern,
-            List<String> parameterValues) {
-        PName tenantId = connection.getTenantId();
-        if (tenantIdPattern == null) {
-            if (tenantId != null) {
-                appendConjunction(buf);
-                buf.append(" (" + TENANT_ID + " IS NULL " +
-                        " OR " + TENANT_ID + " = ?) ");
-                parameterValues.add(tenantId.getString());
-            }
-        } else if (tenantIdPattern.length() == 0) {
-                appendConjunction(buf);
-                buf.append(TENANT_ID + " IS NULL ");
-        } else {
-            appendConjunction(buf);
-            buf.append(" TENANT_ID LIKE ? ");
-            parameterValues.add(tenantIdPattern);
-            if (tenantId != null) {
-                buf.append(" and TENANT_ID = ? ");
-                parameterValues.add(tenantId.getString());
-            }
-        }
-    }
 
     private static void appendConjunction(StringBuilder buf) {
         buf.append(buf.length() == 0 ? "" : " and ");
@@ -1031,50 +986,14 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     }
 
     @Override
-    public ResultSet getIndexInfo(String catalog, String schema, String table, boolean unique, boolean approximate)
-            throws SQLException {
-        if (unique) { // No unique indexes
-            return emptyResultSet;
-        }
-        List<String> parameterValues = new ArrayList<String>(4);
-        StringBuilder buf = new StringBuilder("select \n" +
-                TENANT_ID + " " + TABLE_CAT + ",\n" + // use this column for column family name
-                TABLE_SCHEM + ",\n" +
-                DATA_TABLE_NAME + " " + TABLE_NAME + ",\n" +
-                "true NON_UNIQUE,\n" +
-                "null INDEX_QUALIFIER,\n" +
-                TABLE_NAME + " INDEX_NAME,\n" +
-                DatabaseMetaData.tableIndexOther + " TYPE,\n" +
-                ORDINAL_POSITION + ",\n" +
-                COLUMN_NAME + ",\n" +
-                "CASE WHEN " + COLUMN_FAMILY + " IS NOT NULL THEN null WHEN " + SORT_ORDER + " = " + (SortOrder.DESC.getSystemValue()) + " THEN 'D' ELSE 'A' END ASC_OR_DESC,\n" +
-                "null CARDINALITY,\n" +
-                "null PAGES,\n" +
-                "null FILTER_CONDITION,\n" +
-                // Include data type info, though not in spec
-                ExternalSqlTypeIdFunction.NAME + "(" + DATA_TYPE + ") AS " + DATA_TYPE + ",\n" +
-                SqlTypeNameFunction.NAME + "(" + DATA_TYPE + ") AS " + TYPE_NAME + ",\n" +
-                DATA_TYPE + " " + TYPE_ID + ",\n" +
-                COLUMN_FAMILY + ",\n" +
-                COLUMN_SIZE + ",\n" +
-                ARRAY_SIZE +
-                "\nfrom " + SYSTEM_CATALOG +
-                "\nwhere ");
-        buf.append(TABLE_SCHEM + (schema == null || schema.length() == 0 ? " is null" : " = ?" ));
-        if(schema != null && schema.length() > 0) {
-            parameterValues.add(schema);
-        }
-        buf.append("\nand " + DATA_TABLE_NAME + " = ?" );
-        parameterValues.add(table);
-        buf.append("\nand " + COLUMN_NAME + " is not null" );
-        addTenantIdFilter(buf, catalog, parameterValues);
-        buf.append("\norder by INDEX_NAME," + ORDINAL_POSITION);
-        PreparedStatement stmt = connection.prepareStatement(buf.toString());
-        setParameters(stmt, parameterValues);
+    public ResultSet getIndexInfo(String catalog, String schema, String table, boolean unique,
+        boolean approximate) throws SQLException {
+        PreparedStatement stmt = QueryUtil.getIndexInfoStmt(connection, catalog, schema, table,
+            unique, approximate);
+        if (stmt == null) return emptyResultSet;
         return stmt.executeQuery();
     }
 
-
     @Override
     public int getJDBCMajorVersion() throws SQLException {
         return 1;
@@ -1330,26 +1249,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
 
     @Override
     public ResultSet getSchemas(String catalog, String schemaPattern) throws SQLException {
-        List<String> parameterValues = new ArrayList<String>(4);
-        StringBuilder buf = new StringBuilder("select distinct \n" +
-                TABLE_SCHEM + "," +
-                TENANT_ID + " " + TABLE_CATALOG +
-                " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS +
-                " where " + COLUMN_NAME + " is null");
-        addTenantIdFilter(buf, catalog, parameterValues);
-        if (schemaPattern != null) {
-            buf.append(" and " + TABLE_SCHEM + " like ?");
-            parameterValues.add(schemaPattern);
-        }
-        if (SchemaUtil.isNamespaceMappingEnabled(null, connection.getQueryServices().getProps())) {
-            buf.append(" and " + TABLE_NAME + " = '" + MetaDataClient.EMPTY_TABLE + "'");
-        }
-
-        // TODO: we should union this with SYSTEM.SEQUENCE too, but we only have support for
-        // UNION ALL and we really need UNION so that it dedups.
-
-        PreparedStatement stmt = connection.prepareStatement(buf.toString());
-        setParameters(stmt, parameterValues);
+        PreparedStatement stmt = QueryUtil.getSchemasStmt(connection, catalog, schemaPattern);
         return stmt.executeQuery();
     }
 
@@ -1365,30 +1265,10 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
 
     @Override
     // TODO does this need to change to use the PARENT_TABLE link
-    public ResultSet getSuperTables(String catalog, String schemaPattern, String tableNamePattern) throws SQLException {
-        List<String> parameterValues = new ArrayList<String>(4);
-        StringBuilder buf = new StringBuilder("select \n" +
-                TENANT_ID + " " + TABLE_CAT + "," + // Use tenantId for catalog
-                TABLE_SCHEM + "," +
-                TABLE_NAME + "," +
-                COLUMN_FAMILY + " " + SUPERTABLE_NAME +
-                " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS +
-                " where " + COLUMN_NAME + " is null" +
-                " and " + LINK_TYPE + " = " + LinkType.PHYSICAL_TABLE.getSerializedValue());
-        addTenantIdFilter(buf, catalog, parameterValues);
-        if (schemaPattern != null) {
-            buf.append(" and " + TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like ?" ));
-            if(schemaPattern.length() > 0) {
-                parameterValues.add(schemaPattern);
-            }
-        }
-        if (tableNamePattern != null) {
-            buf.append(" and " + TABLE_NAME + " like ?" );
-            parameterValues.add(tableNamePattern);
-        }
-        buf.append(" order by " + TENANT_ID + "," + TABLE_SCHEM + "," +TABLE_NAME + "," + SUPERTABLE_NAME);
-        PreparedStatement stmt = connection.prepareStatement(buf.toString());
-        setParameters(stmt, parameterValues);
+    public ResultSet getSuperTables(String catalog, String schemaPattern, String tableNamePattern)
+        throws SQLException {
+        PreparedStatement stmt = QueryUtil.getSuperTablesStmt(connection, catalog, schemaPattern,
+            tableNamePattern);
         return stmt.executeQuery();
     }
 
@@ -1458,129 +1338,11 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     }
 
     @Override
-    public ResultSet getTables(String catalog, String schemaPattern, String tableNamePattern, String[] types)
-            throws SQLException {
-        boolean isSequence = false;
-        boolean hasTableTypes = types != null && types.length > 0;
-        StringBuilder typeClauseBuf = new StringBuilder();
-        List<String> parameterValues = new ArrayList<String>(4);
-        if (hasTableTypes) {
-            List<String> tableTypes = Lists.newArrayList(types);
-            isSequence = tableTypes.remove(SEQUENCE_TABLE_TYPE);
-            StringBuilder typeBuf = new StringBuilder();
-            for (String type : tableTypes) {
-                try {
-                    PTableType tableType = PTableType.fromValue(type);
-                    typeBuf.append('\'');
-                    typeBuf.append(tableType.getSerializedValue());
-                    typeBuf.append('\'');
-                    typeBuf.append(',');
-                } catch (IllegalArgumentException e) {
-                    // Ignore and continue
-                }
-            }
-            if (typeBuf.length() > 0) {
-                typeClauseBuf.append(" and " + TABLE_TYPE + " IN (");
-                typeClauseBuf.append(typeBuf);
-                typeClauseBuf.setCharAt(typeClauseBuf.length()-1, ')');
-            }
-        }
-        StringBuilder buf = new StringBuilder("select \n");
-        // If there were table types specified and they were all filtered out
-        // and we're not querying for sequences, return an empty result set.
-        if (hasTableTypes && typeClauseBuf.length() == 0 && !isSequence) {
-            return this.emptyResultSet;
-        }
-        if (typeClauseBuf.length() > 0 || !isSequence) {
-            buf.append(
-                    TENANT_ID + " " + TABLE_CAT + "," + // tenant_id is the catalog
-                    TABLE_SCHEM + "," +
-                    TABLE_NAME + " ," +
-                    SQLTableTypeFunction.NAME + "(" + TABLE_TYPE + ") AS " + TABLE_TYPE + "," +
-                    REMARKS + " ," +
-                    TYPE_NAME + "," +
-                    SELF_REFERENCING_COL_NAME + "," +
-                    REF_GENERATION + "," +
-                    IndexStateNameFunction.NAME + "(" + INDEX_STATE + ") AS " + INDEX_STATE + "," +
-                     IMMUTABLE_ROWS + "," +
-                    SALT_BUCKETS + "," +
-                    MULTI_TENANT + "," +
-                    VIEW_STATEMENT + "," +
-                    SQLViewTypeFunction.NAME + "(" + VIEW_TYPE + ") AS " + VIEW_TYPE + "," +
-                    SQLIndexTypeFunction.NAME + "(" + INDEX_TYPE + ") AS " + INDEX_TYPE + "," +
-                    TRANSACTION_PROVIDER + " IS NOT NULL AS " + TRANSACTIONAL + "," +
-                    IS_NAMESPACE_MAPPED + "," +
-                    GUIDE_POSTS_WIDTH + "," +
-                    TransactionProviderNameFunction.NAME + "(" + TRANSACTION_PROVIDER + ") AS TRANSACTION_PROVIDER" +
-                    " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS +
-                    " where " + COLUMN_NAME + " is null" +
-                    " and " + COLUMN_FAMILY + " is null" +
-                    " and " + TABLE_NAME + " != '" + MetaDataClient.EMPTY_TABLE + "'");
-            addTenantIdFilter(buf, catalog, parameterValues);
-            if (schemaPattern != null) {
-                buf.append(" and " + TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like ?" ));
-                if(schemaPattern.length() > 0) {
-                    parameterValues.add(schemaPattern);
-                }
-            }
-            if (tableNamePattern != null) {
-                buf.append(" and " + TABLE_NAME + " like ?" );
-                parameterValues.add(tableNamePattern);
-            }
-            if (typeClauseBuf.length() > 0) {
-                buf.append(typeClauseBuf);
-            }
-        }
-        if (isSequence) {
-            // Union the SYSTEM.CATALOG entries with the SYSTEM.SEQUENCE entries
-            if (typeClauseBuf.length() > 0) {
-                buf.append(" UNION ALL\n");
-                buf.append(" select\n");
-            }
-            buf.append(
-                    TENANT_ID + " " + TABLE_CAT + "," + // tenant_id is the catalog
-                    SEQUENCE_SCHEMA + " " + TABLE_SCHEM + "," +
-                    SEQUENCE_NAME + " " + TABLE_NAME + " ," +
-                    "'" + SEQUENCE_TABLE_TYPE + "' " + TABLE_TYPE + "," +
-                    "'' " + REMARKS + " ," +
-                    "'' " + TYPE_NAME + "," +
-                    "'' " + SELF_REFERENCING_COL_NAME + "," +
-                    "'' " + REF_GENERATION + "," +
-                    "CAST(null AS CHAR(1)) " + INDEX_STATE + "," +
-                    "CAST(null AS BOOLEAN) " + IMMUTABLE_ROWS + "," +
-                    "CAST(null AS INTEGER) " + SALT_BUCKETS + "," +
-                    "CAST(null AS BOOLEAN) " + MULTI_TENANT + "," +
-                    "'' " + VIEW_STATEMENT + "," +
-                    "'' " + VIEW_TYPE + "," +
-                    "'' " + INDEX_TYPE + "," +
-                    "CAST(null AS BOOLEAN) " + TRANSACTIONAL + "," +
-                    "CAST(null AS BOOLEAN) " + IS_NAMESPACE_MAPPED + "," +
-                    "CAST(null AS BIGINT) " + GUIDE_POSTS_WIDTH + "," +
-                    "CAST(null AS VARCHAR) " + TRANSACTION_PROVIDER + "\n");
-            buf.append(
-                    " from " + SYSTEM_SEQUENCE + "\n");
-            StringBuilder whereClause = new StringBuilder();
-            addTenantIdFilter(whereClause, catalog, parameterValues);
-            if (schemaPattern != null) {
-                appendConjunction(whereClause);
-                whereClause.append(SEQUENCE_SCHEMA + (schemaPattern.length() == 0 ? " is null" : " like ?\n" ));
-                if(schemaPattern.length() > 0) {
-                    parameterValues.add(schemaPattern);
-                }
-            }
-            if (tableNamePattern != null) {
-                appendConjunction(whereClause);
-                whereClause.append(SEQUENCE_NAME + " like ?\n" );
-                parameterValues.add(tableNamePattern);
-            }
-            if (whereClause.length() > 0) {
-                buf.append(" where\n");
-                buf.append(whereClause);
-            }
-        }
-        buf.append(" order by 4, 1, 2, 3\n");
-        PreparedStatement stmt = connection.prepareStatement(buf.toString());
-        setParameters(stmt, parameterValues);
+    public ResultSet getTables(String catalog, String schemaPattern, String tableNamePattern,
+        String[] types) throws SQLException {
+        PreparedStatement stmt = QueryUtil.getTablesStmt(connection, catalog, schemaPattern,
+            tableNamePattern, types);
+        if (stmt == null) return emptyResultSet;
         return stmt.executeQuery();
     }
 
@@ -2100,12 +1862,4 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public boolean generatedKeyAlwaysReturned() throws SQLException {
         return false;
     }
-
-
-    private void setParameters(PreparedStatement stmt, List<String> parameterValues)
-            throws SQLException {
-        for(int i = 0; i < parameterValues.size(); i++) {
-            stmt.setString(i+1, parameterValues.get(i));
-        }
-    }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index af4ae47..ff841be4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -25,6 +25,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.Reader;
 import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
@@ -145,6 +146,8 @@ import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.PrimaryKeyConstraint;
 import org.apache.phoenix.parse.SQLParser;
 import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.parse.ShowSchemasStatement;
+import org.apache.phoenix.parse.ShowTablesStatement;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.parse.TableNode;
 import org.apache.phoenix.parse.TraceStatement;
@@ -1102,6 +1105,35 @@ public class PhoenixStatement implements Statement, SQLCloseable {
         }
     }
 
+    private static class ExecutableShowTablesStatement extends ShowTablesStatement
+        implements CompilableStatement {
+
+        public ExecutableShowTablesStatement(String schema, String pattern) {
+          super(schema, pattern);
+        }
+
+        @Override
+        public QueryPlan compilePlan(final PhoenixStatement stmt, Sequence.ValueOp seqAction)
+            throws SQLException {
+            PreparedStatement delegateStmt = QueryUtil.getTablesStmt(stmt.getConnection(), null,
+                getTargetSchema(), getDbPattern(), null);
+            return ((PhoenixPreparedStatement) delegateStmt).compileQuery();
+        }
+    }
+
+    // Delegates to a SELECT query against SYSCAT.
+    private static class ExecutableShowSchemasStatement extends ShowSchemasStatement implements CompilableStatement {
+
+        public ExecutableShowSchemasStatement(String pattern) { super(pattern); }
+
+        @Override
+        public QueryPlan compilePlan(final PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
+            PreparedStatement delegateStmt =
+                QueryUtil.getSchemasStmt(stmt.getConnection(), null, getSchemaPattern());
+            return ((PhoenixPreparedStatement) delegateStmt).compileQuery();
+        }
+    }
+
     private static class ExecutableCreateIndexStatement extends CreateIndexStatement implements CompilableStatement {
 
         public ExecutableCreateIndexStatement(NamedNode indexName, NamedTableNode dataTable, IndexKeyConstraint ikConstraint, List<ColumnName> includeColumns, List<ParseNode> splits,
@@ -1679,6 +1711,16 @@ public class PhoenixStatement implements Statement, SQLCloseable {
             return new ExecutableChangePermsStatement(permsString, isSchemaName, tableName, schemaName, isGroupName, userOrGroup,isGrantStatement);
         }
 
+        @Override
+        public ShowTablesStatement showTablesStatement(String schema, String pattern) {
+            return new ExecutableShowTablesStatement(schema, pattern);
+        }
+
+        @Override
+        public ShowSchemasStatement showSchemasStatement(String pattern) {
+            return new ExecutableShowSchemasStatement(pattern);
+        }
+
     }
     
     static class PhoenixStatementParser extends SQLParser {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
index 84e7998..f60e18f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -934,4 +934,11 @@ public class ParseNodeFactory {
         return new ChangePermsStatement(permsString, isSchemaName, tableName, schemaName, isGroupName, userOrGroup, isGrantStatement);
     }
 
+    public ShowTablesStatement showTablesStatement(String schema, String pattern) {
+        return new ShowTablesStatement(schema, pattern);
+    }
+
+    public ShowSchemasStatement showSchemasStatement(String pattern) {
+        return new ShowSchemasStatement(pattern);
+    }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ShowSchemasStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ShowSchemasStatement.java
new file mode 100644
index 0000000..da42d56
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ShowSchemasStatement.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.parse;
+
+import com.google.common.base.Preconditions;
+import org.apache.phoenix.compile.ColumnResolver;
+
+import javax.annotation.Nullable;
+import java.util.Objects;
+
+/**
+ * ParseNode implementation for SHOW SCHEMAS sql.
+ */
+public class ShowSchemasStatement extends ShowStatement {
+    @Nullable
+    private final String schemaPattern;
+
+    public ShowSchemasStatement(String pattern) {
+        schemaPattern = pattern;
+    };
+
+    @Nullable
+    protected String getSchemaPattern() {
+        return schemaPattern;
+    }
+
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        Preconditions.checkNotNull(buf);
+        buf.append("SHOW SCHEMAS");
+        if (schemaPattern != null) {
+            buf.append(" LIKE ");
+            buf.append("'").append(schemaPattern).append("'");
+        }
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder();
+        toSQL(null, buf);
+        return buf.toString();
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (!(other instanceof ShowSchemasStatement)) return false;
+        ShowSchemasStatement stmt = (ShowSchemasStatement) other;
+        return Objects.equals(schemaPattern, stmt.getSchemaPattern());
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(schemaPattern);
+    }
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ShowStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ShowStatement.java
new file mode 100644
index 0000000..d4ab7a4
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ShowStatement.java
@@ -0,0 +1,38 @@
+/*
+ * 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.phoenix.parse;
+
+import org.apache.phoenix.jdbc.PhoenixStatement;
+
+/**
+ * Parent class for all SHOW statements. SHOW SCHEMAS, SHOW TABLES etc.
+ */
+public class ShowStatement implements BindableStatement {
+    @Override
+    public int getBindCount() {
+        return 0;
+    }
+
+    @Override
+    public PhoenixStatement.Operation getOperation() {
+        return PhoenixStatement.Operation.QUERY;
+    }
+
+    public ShowStatement () {}
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ShowTablesStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ShowTablesStatement.java
new file mode 100644
index 0000000..b0cde12
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ShowTablesStatement.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.parse;
+
+import com.google.common.base.Preconditions;
+import org.apache.phoenix.compile.ColumnResolver;
+
+import javax.annotation.Nullable;
+import java.util.Objects;
+
+/**
+ * ParseNode implementation for SHOW TABLES [IN <schema>].
+ */
+public class ShowTablesStatement extends ShowStatement {
+    // Schema for table listing. null implies the the db for this connection is used.
+    @Nullable
+    private String targetSchema;
+
+    // Pattern to be matched while looking up for tables in 'targetSchema'.
+    // null implies everything is returned.
+    @Nullable
+    private String dbPattern;
+
+    public  ShowTablesStatement() {
+        this(null, null);
+    }
+
+    public ShowTablesStatement(@Nullable String schema, @Nullable String pattern) {
+        targetSchema = schema;
+        dbPattern = pattern;
+    }
+
+    @Nullable
+    public String getTargetSchema() {
+        return targetSchema;
+    }
+
+    @Nullable
+    public String getDbPattern() {
+        return dbPattern;
+    }
+
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        Preconditions.checkNotNull(buf);
+        buf.append("SHOW TABLES");
+        if (targetSchema != null) {
+            buf.append(" IN ");
+            buf.append(targetSchema);
+            buf.append(" ");
+        }
+        if (dbPattern != null) {
+            buf.append(" LIKE ");
+            buf.append("'").append(dbPattern).append("'");
+        }
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder();
+        toSQL(null, buf);
+        return buf.toString();
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (!(other instanceof ShowTablesStatement)) return false;
+        ShowTablesStatement stmt = (ShowTablesStatement) other;
+        return Objects.equals(targetSchema, stmt.getTargetSchema()) && Objects.equals(dbPattern,
+            stmt.getDbPattern());
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(targetSchema, dbPattern);
+    }
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
index aa3c749..c4c1ab3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
@@ -18,13 +18,53 @@
 
 package org.apache.phoenix.util;
 
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.REF_GENERATION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.REMARKS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SELF_REFERENCING_COL_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SEQUENCE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SEQUENCE_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SEQUENCE_TABLE_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SUPERTABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_ALIAS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_SEQUENCE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_CAT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_CATALOG;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTIONAL;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTION_PROVIDER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_ID;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
 import static org.apache.phoenix.util.SchemaUtil.getEscapedFullColumnName;
 
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
@@ -35,6 +75,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.phoenix.expression.function.ExternalSqlTypeIdFunction;
+import org.apache.phoenix.expression.function.IndexStateNameFunction;
+import org.apache.phoenix.expression.function.SQLIndexTypeFunction;
+import org.apache.phoenix.expression.function.SQLTableTypeFunction;
+import org.apache.phoenix.expression.function.SQLViewTypeFunction;
+import org.apache.phoenix.expression.function.SqlTypeNameFunction;
+import org.apache.phoenix.expression.function.TransactionProviderNameFunction;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -45,6 +92,10 @@ import org.apache.phoenix.parse.WildcardParseNode;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PInteger;
@@ -463,5 +514,285 @@ public final class QueryUtil {
         //we don't need this connection to upgrade anything or start dispatcher
         return getConnectionOnServer(config);
     }
+
+    public static PreparedStatement getCatalogsStmt(PhoenixConnection connection) throws SQLException {
+        List<String> parameterValues = new ArrayList<String>(4);
+        StringBuilder buf = new StringBuilder("select \n" +
+            " DISTINCT " + TENANT_ID + " " + TABLE_CAT +
+            " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS +
+            " where " + COLUMN_NAME + " is null" +
+            " and " + COLUMN_FAMILY + " is null" +
+            " and " + TENANT_ID + " is not null");
+        addTenantIdFilter(connection, buf, null, parameterValues);
+        buf.append(" order by " + TENANT_ID);
+        PreparedStatement stmt = connection.prepareStatement(buf.toString());
+        for(int i = 0; i < parameterValues.size(); i++) {
+            stmt.setString(i+1, parameterValues.get(i));
+        }
+        return stmt;
+    }
+
+    /**
+     * Util that generates a PreparedStatement against syscat to fetch schema listings.
+     */
+    public static PreparedStatement getSchemasStmt(
+        PhoenixConnection connection, String catalog, String schemaPattern) throws SQLException {
+        List<String> parameterValues = new ArrayList<String>(4);
+        StringBuilder buf = new StringBuilder("select distinct \n" +
+                TABLE_SCHEM + "," +
+                TENANT_ID + " " + TABLE_CATALOG +
+                " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS +
+                " where " + COLUMN_NAME + " is null");
+        addTenantIdFilter(connection, buf, catalog, parameterValues);
+        if (schemaPattern != null) {
+            buf.append(" and " + TABLE_SCHEM + " like ?");
+            parameterValues.add(schemaPattern);
+        }
+        if (SchemaUtil.isNamespaceMappingEnabled(null, connection.getQueryServices().getProps())) {
+            buf.append(" and " + TABLE_NAME + " = '" + MetaDataClient.EMPTY_TABLE + "'");
+        }
+
+        // TODO: we should union this with SYSTEM.SEQUENCE too, but we only have support for
+        // UNION ALL and we really need UNION so that it dedups.
+
+        PreparedStatement stmt = connection.prepareStatement(buf.toString());
+        for(int i = 0; i < parameterValues.size(); i++) {
+            stmt.setString(i+1, parameterValues.get(i));
+        }
+        return stmt;
+    }
+
+    public static PreparedStatement getSuperTablesStmt(PhoenixConnection connection,
+        String catalog, String schemaPattern, String tableNamePattern) throws SQLException {
+        List<String> parameterValues = new ArrayList<String>(4);
+        StringBuilder buf = new StringBuilder("select \n" +
+                TENANT_ID + " " + TABLE_CAT + "," + // Use tenantId for catalog
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                COLUMN_FAMILY + " " + SUPERTABLE_NAME +
+                " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS +
+                " where " + COLUMN_NAME + " is null" +
+                " and " + LINK_TYPE + " = " + PTable.LinkType.PHYSICAL_TABLE.getSerializedValue());
+        addTenantIdFilter(connection, buf, catalog, parameterValues);
+        if (schemaPattern != null) {
+            buf.append(" and " + TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like ?" ));
+            if(schemaPattern.length() > 0) {
+                parameterValues.add(schemaPattern);
+            }
+        }
+        if (tableNamePattern != null) {
+            buf.append(" and " + TABLE_NAME + " like ?" );
+            parameterValues.add(tableNamePattern);
+        }
+        buf.append(" order by " + TENANT_ID + "," + TABLE_SCHEM + "," +TABLE_NAME + "," + SUPERTABLE_NAME);
+        PreparedStatement stmt = connection.prepareStatement(buf.toString());
+        for(int i = 0; i < parameterValues.size(); i++) {
+            stmt.setString(i+1, parameterValues.get(i));
+        }
+        return stmt;
+    }
+
+    public static PreparedStatement getIndexInfoStmt(PhoenixConnection connection,
+            String catalog, String schema, String table, boolean unique, boolean approximate) throws SQLException {
+        if (unique) { // No unique indexes
+            return null;
+        }
+        List<String> parameterValues = new ArrayList<String>(4);
+        StringBuilder buf = new StringBuilder("select \n" +
+            TENANT_ID + " " + TABLE_CAT + ",\n" + // use this column for column family name
+            TABLE_SCHEM + ",\n" +
+            DATA_TABLE_NAME + " " + TABLE_NAME + ",\n" +
+            "true NON_UNIQUE,\n" +
+            "null INDEX_QUALIFIER,\n" +
+            TABLE_NAME + " INDEX_NAME,\n" +
+            DatabaseMetaData.tableIndexOther + " TYPE,\n" +
+            ORDINAL_POSITION + ",\n" +
+            COLUMN_NAME + ",\n" +
+            "CASE WHEN " + COLUMN_FAMILY + " IS NOT NULL THEN null WHEN " + SORT_ORDER + " = " + (SortOrder.DESC.getSystemValue()) + " THEN 'D' ELSE 'A' END ASC_OR_DESC,\n" +
+            "null CARDINALITY,\n" +
+            "null PAGES,\n" +
+            "null FILTER_CONDITION,\n" +
+            // Include data type info, though not in spec
+            ExternalSqlTypeIdFunction.NAME + "(" + DATA_TYPE + ") AS " + DATA_TYPE + ",\n" +
+            SqlTypeNameFunction.NAME + "(" + DATA_TYPE + ") AS " + TYPE_NAME + ",\n" +
+            DATA_TYPE + " " + TYPE_ID + ",\n" +
+            COLUMN_FAMILY + ",\n" +
+            COLUMN_SIZE + ",\n" +
+            ARRAY_SIZE +
+            "\nfrom " + SYSTEM_CATALOG +
+            "\nwhere ");
+        buf.append(TABLE_SCHEM + (schema == null || schema.length() == 0 ? " is null" : " = ?" ));
+        if(schema != null && schema.length() > 0) {
+            parameterValues.add(schema);
+        }
+        buf.append("\nand " + DATA_TABLE_NAME + " = ?" );
+        parameterValues.add(table);
+        buf.append("\nand " + COLUMN_NAME + " is not null" );
+        addTenantIdFilter(connection, buf, catalog, parameterValues);
+        buf.append("\norder by INDEX_NAME," + ORDINAL_POSITION);
+        PreparedStatement stmt = connection.prepareStatement(buf.toString());
+        for(int i = 0; i < parameterValues.size(); i++) {
+            stmt.setString(i+1, parameterValues.get(i));
+        }
+        return stmt;
+    }
+
+    /**
+     * Util that generates a PreparedStatement against syscat to get the table listing in a given schema.
+     */
+    public static PreparedStatement getTablesStmt(PhoenixConnection connection, String catalog, String schemaPattern,
+        String tableNamePattern, String[] types) throws SQLException {
+        boolean isSequence = false;
+        boolean hasTableTypes = types != null && types.length > 0;
+        StringBuilder typeClauseBuf = new StringBuilder();
+        List<String> parameterValues = new ArrayList<String>(4);
+        if (hasTableTypes) {
+            List<String> tableTypes = Lists.newArrayList(types);
+            isSequence = tableTypes.remove(SEQUENCE_TABLE_TYPE);
+            StringBuilder typeBuf = new StringBuilder();
+            for (String type : tableTypes) {
+                try {
+                    PTableType tableType = PTableType.fromValue(type);
+                    typeBuf.append('\'');
+                    typeBuf.append(tableType.getSerializedValue());
+                    typeBuf.append('\'');
+                    typeBuf.append(',');
+                } catch (IllegalArgumentException e) {
+                    // Ignore and continue
+                }
+            }
+            if (typeBuf.length() > 0) {
+                typeClauseBuf.append(" and " + TABLE_TYPE + " IN (");
+                typeClauseBuf.append(typeBuf);
+                typeClauseBuf.setCharAt(typeClauseBuf.length()-1, ')');
+            }
+        }
+        StringBuilder buf = new StringBuilder("select \n");
+        // If there were table types specified and they were all filtered out
+        // and we're not querying for sequences, return an empty result set.
+        if (hasTableTypes && typeClauseBuf.length() == 0 && !isSequence) {
+            return null;
+        }
+        if (typeClauseBuf.length() > 0 || !isSequence) {
+            buf.append(
+                TENANT_ID + " " + TABLE_CAT + "," + // tenant_id is the catalog
+                TABLE_SCHEM + "," +
+                TABLE_NAME + " ," +
+                SQLTableTypeFunction.NAME + "(" + TABLE_TYPE + ") AS " + TABLE_TYPE + "," +
+                REMARKS + " ," +
+                TYPE_NAME + "," +
+                SELF_REFERENCING_COL_NAME + "," +
+                REF_GENERATION + "," +
+                IndexStateNameFunction.NAME + "(" + INDEX_STATE + ") AS " + INDEX_STATE + "," +
+                IMMUTABLE_ROWS + "," +
+                SALT_BUCKETS + "," +
+                MULTI_TENANT + "," +
+                VIEW_STATEMENT + "," +
+                SQLViewTypeFunction.NAME + "(" + VIEW_TYPE + ") AS " + VIEW_TYPE + "," +
+                SQLIndexTypeFunction.NAME + "(" + INDEX_TYPE + ") AS " + INDEX_TYPE + "," +
+                TRANSACTION_PROVIDER + " IS NOT NULL AS " + TRANSACTIONAL + "," +
+                IS_NAMESPACE_MAPPED + "," +
+                GUIDE_POSTS_WIDTH + "," +
+                TransactionProviderNameFunction.NAME + "(" + TRANSACTION_PROVIDER + ") AS TRANSACTION_PROVIDER" +
+                " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS +
+                " where " + COLUMN_NAME + " is null" +
+                " and " + COLUMN_FAMILY + " is null" +
+                " and " + TABLE_NAME + " != '" + MetaDataClient.EMPTY_TABLE + "'");
+            addTenantIdFilter(connection, buf, catalog, parameterValues);
+            if (schemaPattern != null) {
+                buf.append(" and " + TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like ?" ));
+                if(schemaPattern.length() > 0) {
+                    parameterValues.add(schemaPattern);
+                }
+            }
+            if (tableNamePattern != null) {
+                buf.append(" and " + TABLE_NAME + " like ?" );
+                parameterValues.add(tableNamePattern);
+            }
+            if (typeClauseBuf.length() > 0) {
+                buf.append(typeClauseBuf);
+            }
+        }
+        if (isSequence) {
+            // Union the SYSTEM.CATALOG entries with the SYSTEM.SEQUENCE entries
+            if (typeClauseBuf.length() > 0) {
+                buf.append(" UNION ALL\n");
+                buf.append(" select\n");
+            }
+            buf.append(
+                TENANT_ID + " " + TABLE_CAT + "," + // tenant_id is the catalog
+                SEQUENCE_SCHEMA + " " + TABLE_SCHEM + "," +
+                SEQUENCE_NAME + " " + TABLE_NAME + " ," +
+                "'" + SEQUENCE_TABLE_TYPE + "' " + TABLE_TYPE + "," +
+                "'' " + REMARKS + " ," +
+                "'' " + TYPE_NAME + "," +
+                "'' " + SELF_REFERENCING_COL_NAME + "," +
+                "'' " + REF_GENERATION + "," +
+                "CAST(null AS CHAR(1)) " + INDEX_STATE + "," +
+                "CAST(null AS BOOLEAN) " + IMMUTABLE_ROWS + "," +
+                "CAST(null AS INTEGER) " + SALT_BUCKETS + "," +
+                "CAST(null AS BOOLEAN) " + MULTI_TENANT + "," +
+                "'' " + VIEW_STATEMENT + "," +
+                "'' " + VIEW_TYPE + "," +
+                "'' " + INDEX_TYPE + "," +
+                "CAST(null AS BOOLEAN) " + TRANSACTIONAL + "," +
+                "CAST(null AS BOOLEAN) " + IS_NAMESPACE_MAPPED + "," +
+                "CAST(null AS BIGINT) " + GUIDE_POSTS_WIDTH + "," +
+                "CAST(null AS VARCHAR) " + TRANSACTION_PROVIDER + "\n");
+            buf.append(" from " + SYSTEM_SEQUENCE + "\n");
+            StringBuilder whereClause = new StringBuilder();
+            addTenantIdFilter(connection, whereClause, catalog, parameterValues);
+            if (schemaPattern != null) {
+                appendConjunction(whereClause);
+                whereClause.append(SEQUENCE_SCHEMA + (schemaPattern.length() == 0 ? " is null" : " like ?\n" ));
+                if(schemaPattern.length() > 0) {
+                    parameterValues.add(schemaPattern);
+                }
+            }
+            if (tableNamePattern != null) {
+                appendConjunction(whereClause);
+                whereClause.append(SEQUENCE_NAME + " like ?\n" );
+                parameterValues.add(tableNamePattern);
+            }
+            if (whereClause.length() > 0) {
+                buf.append(" where\n");
+                buf.append(whereClause);
+            }
+        }
+        buf.append(" order by 4, 1, 2, 3\n");
+        PreparedStatement stmt = connection.prepareStatement(buf.toString());
+        for(int i = 0; i < parameterValues.size(); i++) {
+            stmt.setString(i+1, parameterValues.get(i));
+        }
+        return stmt;
+    }
+
+    public static void addTenantIdFilter(PhoenixConnection connection, StringBuilder buf, String tenantIdPattern,
+                                         List<String> parameterValues) {
+        PName tenantId = connection.getTenantId();
+        if (tenantIdPattern == null) {
+            if (tenantId != null) {
+                appendConjunction(buf);
+                buf.append(" (" + TENANT_ID + " IS NULL " +
+                        " OR " + TENANT_ID + " = ?) ");
+                parameterValues.add(tenantId.getString());
+            }
+        } else if (tenantIdPattern.length() == 0) {
+            appendConjunction(buf);
+            buf.append(TENANT_ID + " IS NULL ");
+        } else {
+            appendConjunction(buf);
+            buf.append(" TENANT_ID LIKE ? ");
+            parameterValues.add(tenantIdPattern);
+            if (tenantId != null) {
+                buf.append(" and TENANT_ID = ? ");
+                parameterValues.add(tenantId.getString());
+            }
+        }
+    }
+
+    private static void appendConjunction(StringBuilder buf) {
+        buf.append(buf.length() == 0 ? "" : " and ");
+    }
     
 }
\ No newline at end of file
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
index 3999bed..d644474 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
@@ -26,8 +26,6 @@ import java.io.IOException;
 import java.io.StringReader;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.hbase.util.Pair;
@@ -880,7 +878,25 @@ public class QueryParserTest {
 
     @Test
     public void testLimitRVCOffsetQuery() throws Exception {
-        String sql = "SELECT * FROM T LIMIT 10 OFFSET (A,B,C)=('a','b','c')";
-        parseQuery(sql);
+      String sql = "SELECT * FROM T LIMIT 10 OFFSET (A,B,C)=('a','b','c')";
+      parseQuery(sql);
+    }
+
+    @Test
+    public void testShowStmt() throws Exception {
+        // Happy paths
+        parseQuery("show schemas");
+        parseQuery("show schemas like 'foo%'");
+        parseQuery("show tables");
+        parseQuery("show tables in foo");
+        parseQuery("show tables in foo like 'bar%'");
+        parseQuery("show tables like 'bar%'");
+
+        // Expected failures.
+        parseQueryThatShouldFail("show schemas like foo");
+        parseQueryThatShouldFail("show schemas in foo");
+        parseQueryThatShouldFail("show tables 'foo'");
+        parseQueryThatShouldFail("show tables in 'foo'");
+        parseQueryThatShouldFail("show tables like foo");
     }
 }
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
index d654138..34f45b2 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
@@ -18,20 +18,15 @@
 
 package org.apache.phoenix.pherf.util;
 
-import org.apache.phoenix.mapreduce.index.IndexTool;
 import org.apache.phoenix.mapreduce.index.automation.PhoenixMRJobSubmitter;
 import org.apache.phoenix.pherf.PherfConstants;
 import org.apache.phoenix.pherf.configuration.*;
-import org.apache.phoenix.pherf.jmx.MonitorManager;
-import org.apache.phoenix.pherf.result.DataLoadThreadTime;
 import org.apache.phoenix.pherf.result.DataLoadTimeSummary;
 import org.apache.phoenix.pherf.rules.RulesApplier;
-import org.apache.phoenix.pherf.util.GoogleChartGenerator.Node;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
 import java.sql.*;
 import java.util.ArrayList;
 import java.util.Collections;