You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ve...@apache.org on 2016/10/13 17:13:30 UTC

[4/4] drill git commit: DRILL-4925: Add tableType filter to GetTables metadata query

DRILL-4925: Add tableType filter to GetTables metadata query

- Adding tableType filter to GetTablesReq query (needed for JDBC and ODBC
drivers).
- Fix table type returned by sys and INFORMATION_SCHEMA tables
- Also fixes some protobuf typos to related classes.

this closes #612

Change-Id: If95246a312f6c6d64a88872936f516308874c2d2


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

Branch: refs/heads/master
Commit: d0464ab9eed8ab118d85af3a36ab1024de5e6af3
Parents: 4304817
Author: Laurent Goujon <la...@dremio.com>
Authored: Tue Sep 27 13:37:31 2016 -0700
Committer: vkorukanti <ve...@dremio.com>
Committed: Wed Oct 12 17:55:45 2016 -0700

----------------------------------------------------------------------
 .../apache/drill/exec/client/DrillClient.java   |  22 +-
 .../drill/exec/planner/logical/DrillTable.java  |  24 +-
 .../store/ischema/InfoSchemaDrillTable.java     |  10 +-
 .../exec/store/ischema/InfoSchemaFilter.java    |  28 +-
 .../ischema/InfoSchemaRecordGenerator.java      |  27 +-
 .../exec/store/ischema/InfoSchemaTable.java     |  65 +-
 .../exec/store/ischema/InfoSchemaTableType.java |  15 +-
 .../drill/exec/store/sys/StaticDrillTable.java  |   6 +-
 .../drill/exec/store/sys/SystemTablePlugin.java |  13 +-
 .../exec/work/metadata/MetadataProvider.java    |  65 +-
 .../work/metadata/TestMetadataProvider.java     |  71 +-
 .../drill/jdbc/test/TestJdbcMetadata.java       |   2 +-
 .../drill/exec/proto/SchemaUserProtos.java      |  47 +-
 .../org/apache/drill/exec/proto/UserProtos.java | 980 +++++++++++--------
 .../drill/exec/proto/beans/GetColumnsReq.java   |  22 +-
 .../drill/exec/proto/beans/GetSchemasReq.java   |  22 +-
 .../drill/exec/proto/beans/GetTablesReq.java    |  54 +-
 .../drill/exec/proto/beans/LikeFilter.java      |  22 +-
 protocol/src/main/protobuf/User.proto           |   9 +-
 19 files changed, 935 insertions(+), 569 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/d0464ab9/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index e81a4fb..2c5cb47 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -17,12 +17,10 @@
  */
 package org.apache.drill.exec.client;
 
-import static com.google.common.base.Preconditions.checkState;
 import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
 import static org.apache.drill.exec.proto.UserProtos.QueryResultsMode.STREAM_FULL;
 import static org.apache.drill.exec.proto.UserProtos.RunQuery.newBuilder;
-import io.netty.buffer.DrillBuf;
-import io.netty.channel.EventLoopGroup;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -55,8 +53,8 @@ import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.proto.UserProtos;
 import org.apache.drill.exec.proto.UserProtos.CreatePreparedStatementReq;
 import org.apache.drill.exec.proto.UserProtos.CreatePreparedStatementResp;
-import org.apache.drill.exec.proto.UserProtos.GetCatalogsResp;
 import org.apache.drill.exec.proto.UserProtos.GetCatalogsReq;
+import org.apache.drill.exec.proto.UserProtos.GetCatalogsResp;
 import org.apache.drill.exec.proto.UserProtos.GetColumnsReq;
 import org.apache.drill.exec.proto.UserProtos.GetColumnsResp;
 import org.apache.drill.exec.proto.UserProtos.GetQueryPlanFragments;
@@ -92,6 +90,9 @@ import com.google.common.base.Strings;
 import com.google.common.util.concurrent.AbstractCheckedFuture;
 import com.google.common.util.concurrent.SettableFuture;
 
+import io.netty.buffer.DrillBuf;
+import io.netty.channel.EventLoopGroup;
+
 /**
  * Thin wrapper around a UserClient that handles connect/close and transforms
  * String into ByteBuf.
@@ -463,7 +464,7 @@ public class DrillClient implements Closeable, ConnectionThrottle {
     }
 
     if (schemaNameFilter != null) {
-      reqBuilder.setSchameNameFilter(schemaNameFilter);
+      reqBuilder.setSchemaNameFilter(schemaNameFilter);
     }
 
     return client.send(RpcType.GET_SCHEMAS, reqBuilder.build(), GetSchemasResp.class);
@@ -475,23 +476,28 @@ public class DrillClient implements Closeable, ConnectionThrottle {
    * @param catalogNameFilter Filter on <code>catalog name</code>. Pass null to apply no filter.
    * @param schemaNameFilter Filter on <code>schema name</code>. Pass null to apply no filter.
    * @param tableNameFilter Filter in <code>table name</code>. Pass null to apply no filter.
+   * @param tableTypeFilter Filter in <code>table type</code>. Pass null to apply no filter
    * @return
    */
   public DrillRpcFuture<GetTablesResp> getTables(LikeFilter catalogNameFilter, LikeFilter schemaNameFilter,
-      LikeFilter tableNameFilter) {
+      LikeFilter tableNameFilter, List<String> tableTypeFilter) {
     final GetTablesReq.Builder reqBuilder = GetTablesReq.newBuilder();
     if (catalogNameFilter != null) {
       reqBuilder.setCatalogNameFilter(catalogNameFilter);
     }
 
     if (schemaNameFilter != null) {
-      reqBuilder.setSchameNameFilter(schemaNameFilter);
+      reqBuilder.setSchemaNameFilter(schemaNameFilter);
     }
 
     if (tableNameFilter != null) {
       reqBuilder.setTableNameFilter(tableNameFilter);
     }
 
+    if (tableTypeFilter != null) {
+      reqBuilder.addAllTableTypeFilter(tableTypeFilter);
+    }
+
     return client.send(RpcType.GET_TABLES, reqBuilder.build(), GetTablesResp.class);
   }
 
@@ -512,7 +518,7 @@ public class DrillClient implements Closeable, ConnectionThrottle {
     }
 
     if (schemaNameFilter != null) {
-      reqBuilder.setSchameNameFilter(schemaNameFilter);
+      reqBuilder.setSchemaNameFilter(schemaNameFilter);
     }
 
     if (tableNameFilter != null) {

http://git-wip-us.apache.org/repos/asf/drill/blob/d0464ab9/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
index 1cb83b1..96043d3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
@@ -19,6 +19,8 @@ package org.apache.drill.exec.planner.logical;
 
 import java.io.IOException;
 
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.schema.Schema.TableType;
 import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.Statistics;
@@ -27,31 +29,43 @@ import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.store.StoragePlugin;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.plan.RelOptTable;
 import org.apache.drill.exec.util.ImpersonationUtil;
 
 public abstract class DrillTable implements Table {
 
   private final String storageEngineName;
   private final StoragePluginConfig storageEngineConfig;
+  private final TableType tableType;
   private final Object selection;
   private final StoragePlugin plugin;
   private final String userName;
-
   private GroupScan scan;
 
   /**
-   * Creates a DrillTable instance.
+   * Creates a DrillTable instance for a @{code TableType#Table} table.
    * @param storageEngineName StorageEngine name.
    * @param plugin Reference to StoragePlugin.
    * @param userName Whom to impersonate while reading the contents of the table.
    * @param selection Table contents (type and contents depend on type of StoragePlugin).
    */
   public DrillTable(String storageEngineName, StoragePlugin plugin, String userName, Object selection) {
+    this(storageEngineName, plugin, TableType.TABLE, userName, selection);
+  }
+
+  /**
+   * Creates a DrillTable instance.
+   * @param storageEngineName StorageEngine name.
+   * @param plugin Reference to StoragePlugin.
+   * @param tableType the JDBC table type
+   * @param userName Whom to impersonate while reading the contents of the table.
+   * @param selection Table contents (type and contents depend on type of StoragePlugin).
+   */
+  public DrillTable(String storageEngineName, StoragePlugin plugin, TableType tableType, String userName, Object selection) {
     this.selection = selection;
     this.plugin = plugin;
 
+    this.tableType = tableType;
+
     this.storageEngineConfig = plugin.getConfig();
     this.storageEngineName = storageEngineName;
     this.userName = userName;
@@ -106,7 +120,7 @@ public abstract class DrillTable implements Table {
 
   @Override
   public TableType getJdbcTableType() {
-    return TableType.TABLE;
+    return tableType;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/d0464ab9/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaDrillTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaDrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaDrillTable.java
index 9c03bcf..48651b1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaDrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaDrillTable.java
@@ -17,18 +17,20 @@
  */
 package org.apache.drill.exec.store.ischema;
 
-import org.apache.drill.common.logical.StoragePluginConfig;
-import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.Schema.TableType;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.util.ImpersonationUtil;
 
-public class InfoSchemaDrillTable extends DrillTable{
+public class InfoSchemaDrillTable extends DrillTable {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InfoSchemaDrillTable.class);
 
   private final InfoSchemaTableType table;
 
   public InfoSchemaDrillTable(InfoSchemaStoragePlugin plugin, String storageEngineName, InfoSchemaTableType selection, StoragePluginConfig storageEngineConfig) {
-    super(storageEngineName, plugin, selection);
+    super(storageEngineName, plugin, TableType.SYSTEM_TABLE, ImpersonationUtil.getProcessUserName(), selection);
     this.table = selection;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/d0464ab9/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilter.java
index 4197a26..22ad7f6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilter.java
@@ -19,17 +19,17 @@ package org.apache.drill.exec.store.ischema;
 
 import static org.apache.drill.exec.expr.fn.impl.RegexpUtil.sqlToRegexLike;
 
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import org.apache.drill.exec.store.ischema.InfoSchemaFilter.ExprNode.Type;
+
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import com.google.common.base.Joiner;
-import org.apache.drill.exec.expr.fn.impl.RegexpUtil;
-import org.apache.drill.exec.store.ischema.InfoSchemaFilter.ExprNode.Type;
-
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Pattern;
 
 @JsonTypeName("info-schema-filter")
 public class InfoSchemaFilter {
@@ -211,6 +211,22 @@ public class InfoSchemaFilter {
 
         return Result.TRUE;
       }
+
+      case "in": {
+        FieldExprNode col = (FieldExprNode) exprNode.args.get(0);
+        List<ExprNode> args = exprNode.args.subList(1, exprNode.args.size());
+        final String fieldValue = recordValues.get(col.field.toString());
+        if (fieldValue != null) {
+          for(ExprNode arg: args) {
+            if (fieldValue.equals(((ConstantExprNode) arg).value)) {
+              return Result.TRUE;
+            }
+          }
+          return Result.FALSE;
+        }
+
+        return Result.INCONCLUSIVE;
+      }
     }
 
     throw new UnsupportedOperationException(

http://git-wip-us.apache.org/repos/asf/drill/blob/d0464ab9/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java
index 5223595..0d31c3c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java
@@ -25,6 +25,7 @@ import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.IS_CATALOG
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SCHS_COL_SCHEMA_NAME;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SHRD_COL_TABLE_NAME;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SHRD_COL_TABLE_SCHEMA;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.TBLS_COL_TABLE_TYPE;
 
 import java.util.List;
 import java.util.Map;
@@ -40,7 +41,6 @@ import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.planner.logical.DrillViewInfoProvider;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.store.AbstractSchema;
-import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.store.ischema.InfoSchemaFilter.Result;
 import org.apache.drill.exec.store.pojo.PojoRecordReader;
 
@@ -54,7 +54,7 @@ import com.google.common.collect.Lists;
  * level specific object is visited and decision is taken to visit the contents of the object. Object here is catalog,
  * schema, table or field.
  */
-public abstract class InfoSchemaRecordGenerator {
+public abstract class InfoSchemaRecordGenerator<S> {
   protected InfoSchemaFilter filter;
 
   protected OptionManager optionManager;
@@ -151,7 +151,7 @@ public abstract class InfoSchemaRecordGenerator {
     return true;
   }
 
-  protected boolean shouldVisitTable(String schemaName, String tableName) {
+  protected boolean shouldVisitTable(String schemaName, String tableName, TableType tableType) {
     if (filter == null) {
       return true;
     }
@@ -161,7 +161,8 @@ public abstract class InfoSchemaRecordGenerator {
             CATS_COL_CATALOG_NAME, IS_CATALOG_NAME,
             SHRD_COL_TABLE_SCHEMA, schemaName,
             SCHS_COL_SCHEMA_NAME, schemaName,
-            SHRD_COL_TABLE_NAME, tableName);
+            SHRD_COL_TABLE_NAME, tableName,
+            TBLS_COL_TABLE_TYPE, tableType.toString());
 
     // If the filter evaluates to false then we don't need to visit the table.
     // For other two results (TRUE, INCONCLUSIVE) continue to visit the table.
@@ -186,7 +187,7 @@ public abstract class InfoSchemaRecordGenerator {
     return filter.evaluate(recordValues) != Result.FALSE;
   }
 
-  public abstract PojoRecordReader<?> getRecordReader();
+  public abstract PojoRecordReader<S> getRecordReader();
 
   public void scanSchema(SchemaPlus root) {
     if (shouldVisitCatalog() && visitCatalog()) {
@@ -225,8 +226,9 @@ public abstract class InfoSchemaRecordGenerator {
     for(Pair<String, ? extends Table> tableNameToTable : drillSchema.getTablesByNames(tableNames)) {
       final String tableName = tableNameToTable.getKey();
       final Table table = tableNameToTable.getValue();
+      final TableType tableType = table.getJdbcTableType();
       // Visit the table, and if requested ...
-      if(shouldVisitTable(schemaPath, tableName) && visitTable(schemaPath,  tableName, table)) {
+      if(shouldVisitTable(schemaPath, tableName, tableType) && visitTable(schemaPath, tableName, table)) {
         // ... do for each of the table's fields.
         final RelDataType tableRow = table.getRowType(new JavaTypeFactoryImpl());
         for (RelDataTypeField field: tableRow.getFieldList()) {
@@ -238,7 +240,7 @@ public abstract class InfoSchemaRecordGenerator {
     }
   }
 
-  public static class Catalogs extends InfoSchemaRecordGenerator {
+  public static class Catalogs extends InfoSchemaRecordGenerator<Records.Catalog> {
     List<Records.Catalog> records = ImmutableList.of();
 
     public Catalogs(OptionManager optionManager) {
@@ -257,7 +259,7 @@ public abstract class InfoSchemaRecordGenerator {
     }
   }
 
-  public static class Schemata extends InfoSchemaRecordGenerator {
+  public static class Schemata extends InfoSchemaRecordGenerator<Records.Schema> {
     List<Records.Schema> records = Lists.newArrayList();
 
     public Schemata(OptionManager optionManager) {
@@ -278,7 +280,7 @@ public abstract class InfoSchemaRecordGenerator {
     }
   }
 
-  public static class Tables extends InfoSchemaRecordGenerator {
+  public static class Tables extends InfoSchemaRecordGenerator<Records.Table> {
     List<Records.Table> records = Lists.newArrayList();
 
     public Tables(OptionManager optionManager) {
@@ -305,8 +307,9 @@ public abstract class InfoSchemaRecordGenerator {
       for(Pair<String, ? extends Table> tableNameToTable : tableNameToTables) {
         final String tableName = tableNameToTable.getKey();
         final Table table = tableNameToTable.getValue();
+        final TableType tableType = tableNameToTable.getValue().getJdbcTableType();
         // Visit the table, and if requested ...
-        if(shouldVisitTable(schemaPath, tableName)) {
+        if(shouldVisitTable(schemaPath, tableName, tableType)) {
           visitTable(schemaPath, tableName, table);
         }
       }
@@ -326,7 +329,7 @@ public abstract class InfoSchemaRecordGenerator {
     }
   }
 
-  public static class Views extends InfoSchemaRecordGenerator {
+  public static class Views extends InfoSchemaRecordGenerator<Records.View> {
     List<Records.View> records = Lists.newArrayList();
 
     public Views(OptionManager optionManager) {
@@ -348,7 +351,7 @@ public abstract class InfoSchemaRecordGenerator {
     }
   }
 
-  public static class Columns extends InfoSchemaRecordGenerator {
+  public static class Columns extends InfoSchemaRecordGenerator<Records.Column> {
     List<Records.Column> records = Lists.newArrayList();
     public Columns(OptionManager optionManager) {
       super(optionManager);

http://git-wip-us.apache.org/repos/asf/drill/blob/d0464ab9/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTable.java
index eb66bc1..925e066 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTable.java
@@ -17,25 +17,56 @@
  */
 package org.apache.drill.exec.store.ischema;
 
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_CONNECT;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_DESCRIPTION;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_NAME;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_CHARACTER_MAXIMUM_LENGTH;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_CHARACTER_OCTET_LENGTH;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_COLUMN_DEFAULT;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_COLUMN_NAME;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_DATA_TYPE;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_DATETIME_PRECISION;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_INTERVAL_PRECISION;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_INTERVAL_TYPE;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_IS_NULLABLE;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_NUMERIC_PRECISION;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_NUMERIC_PRECISION_RADIX;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_NUMERIC_SCALE;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_ORDINAL_POSITION;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SCHS_COL_CATALOG_NAME;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SCHS_COL_IS_MUTABLE;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SCHS_COL_SCHEMA_NAME;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SCHS_COL_SCHEMA_OWNER;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SCHS_COL_TYPE;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SHRD_COL_TABLE_CATALOG;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SHRD_COL_TABLE_NAME;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SHRD_COL_TABLE_SCHEMA;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.TAB_CATALOGS;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.TAB_COLUMNS;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.TAB_SCHEMATA;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.TAB_TABLES;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.TAB_VIEWS;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.TBLS_COL_TABLE_TYPE;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.VIEWS_COL_VIEW_DEFINITION;
+
 import java.util.List;
 
-import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.*;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.common.types.Types;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.server.options.OptionManager;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-import org.apache.drill.exec.server.options.OptionManager;
 
 /**
  * Base class for tables in INFORMATION_SCHEMA.  Defines the table (fields and
  * types).
  */
-public abstract class InfoSchemaTable {
+public abstract class InfoSchemaTable<S> {
 
   public static class Field {
     public String name;
@@ -86,10 +117,10 @@ public abstract class InfoSchemaTable {
     return typeFactory.createStructType(relTypes, fieldNames);
   }
 
-  public abstract InfoSchemaRecordGenerator getRecordGenerator(OptionManager optionManager);
+  public abstract InfoSchemaRecordGenerator<S> getRecordGenerator(OptionManager optionManager);
 
   /** Layout for the CATALOGS table. */
-  static public class Catalogs extends InfoSchemaTable {
+  static public class Catalogs extends InfoSchemaTable<Records.Catalog> {
     // NOTE:  Nothing seems to verify that the types here (apparently used
     // by SQL validation) match the types of the fields in Records.Catalogs).
     private static final List<Field> fields = ImmutableList.of(
@@ -102,13 +133,13 @@ public abstract class InfoSchemaTable {
     }
 
     @Override
-    public InfoSchemaRecordGenerator getRecordGenerator(OptionManager optionManager) {
+    public InfoSchemaRecordGenerator<Records.Catalog> getRecordGenerator(OptionManager optionManager) {
       return new InfoSchemaRecordGenerator.Catalogs(optionManager);
     }
   }
 
   /** Layout for the SCHEMATA table. */
-  public static class Schemata extends InfoSchemaTable {
+  public static class Schemata extends InfoSchemaTable<Records.Schema> {
     // NOTE:  Nothing seems to verify that the types here (apparently used
     // by SQL validation) match the types of the fields in Records.Schemata).
     private static final List<Field> fields = ImmutableList.of(
@@ -123,13 +154,13 @@ public abstract class InfoSchemaTable {
     }
 
     @Override
-    public InfoSchemaRecordGenerator getRecordGenerator(OptionManager optionManager) {
+    public InfoSchemaRecordGenerator<Records.Schema> getRecordGenerator(OptionManager optionManager) {
       return new InfoSchemaRecordGenerator.Schemata(optionManager);
     }
   }
 
   /** Layout for the TABLES table. */
-  public static class Tables extends InfoSchemaTable {
+  public static class Tables extends InfoSchemaTable<Records.Table> {
     // NOTE:  Nothing seems to verify that the types here (apparently used
     // by SQL validation) match the types of the fields in Records.Tables).
     private static final List<Field> fields = ImmutableList.of(
@@ -143,13 +174,13 @@ public abstract class InfoSchemaTable {
     }
 
     @Override
-    public InfoSchemaRecordGenerator getRecordGenerator(OptionManager optionManager) {
+    public InfoSchemaRecordGenerator<Records.Table> getRecordGenerator(OptionManager optionManager) {
       return new InfoSchemaRecordGenerator.Tables(optionManager);
     }
   }
 
   /** Layout for the VIEWS table. */
-  static public class Views extends InfoSchemaTable {
+  static public class Views extends InfoSchemaTable<Records.View> {
     // NOTE:  Nothing seems to verify that the types here (apparently used
     // by SQL validation) match the types of the fields in Records.Views).
     private static final List<Field> fields = ImmutableList.of(
@@ -163,13 +194,13 @@ public abstract class InfoSchemaTable {
     }
 
     @Override
-    public InfoSchemaRecordGenerator getRecordGenerator(OptionManager optionManager) {
+    public InfoSchemaRecordGenerator<Records.View> getRecordGenerator(OptionManager optionManager) {
       return new InfoSchemaRecordGenerator.Views(optionManager);
     }
   }
 
   /** Layout for the COLUMNS table. */
-  public static class Columns extends InfoSchemaTable {
+  public static class Columns extends InfoSchemaTable<Records.Column> {
     // COLUMNS columns, from SQL standard:
     // 1. TABLE_CATALOG
     // 2. TABLE_SCHEMA
@@ -215,7 +246,7 @@ public abstract class InfoSchemaTable {
     }
 
     @Override
-    public InfoSchemaRecordGenerator getRecordGenerator(OptionManager optionManager) {
+    public InfoSchemaRecordGenerator<Records.Column> getRecordGenerator(OptionManager optionManager) {
       return new InfoSchemaRecordGenerator.Columns(optionManager);
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/d0464ab9/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTableType.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTableType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTableType.java
index ec914b2..8f65b66 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTableType.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTableType.java
@@ -17,17 +17,15 @@
  */
 package org.apache.drill.exec.store.ischema;
 
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.SchemaPlus;
-
 import org.apache.drill.exec.server.options.OptionManager;
-import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.store.ischema.InfoSchemaTable.Catalogs;
 import org.apache.drill.exec.store.ischema.InfoSchemaTable.Columns;
 import org.apache.drill.exec.store.ischema.InfoSchemaTable.Schemata;
 import org.apache.drill.exec.store.ischema.InfoSchemaTable.Tables;
 import org.apache.drill.exec.store.ischema.InfoSchemaTable.Views;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.drill.exec.store.pojo.PojoRecordReader;
 
 /**
@@ -43,18 +41,19 @@ public enum InfoSchemaTableType {
   COLUMNS(new Columns()),
   TABLES(new Tables());
 
-  private final InfoSchemaTable tableDef;
+  private final InfoSchemaTable<?> tableDef;
 
   /**
    * ...
    * @param  tableDef  the definition (columns and data generator) of the table
    */
-  InfoSchemaTableType(InfoSchemaTable tableDef) {
+  InfoSchemaTableType(InfoSchemaTable<?> tableDef) {
     this.tableDef = tableDef;
   }
 
-  public PojoRecordReader<?> getRecordReader(SchemaPlus rootSchema, InfoSchemaFilter filter, OptionManager optionManager) {
-    InfoSchemaRecordGenerator recordGenerator = tableDef.getRecordGenerator(optionManager);
+  public <S> PojoRecordReader<S> getRecordReader(SchemaPlus rootSchema, InfoSchemaFilter filter, OptionManager optionManager) {
+    @SuppressWarnings("unchecked")
+    InfoSchemaRecordGenerator<S> recordGenerator = (InfoSchemaRecordGenerator<S>) tableDef.getRecordGenerator(optionManager);
     recordGenerator.setInfoSchemaFilter(filter);
     recordGenerator.scanSchema(rootSchema);
     return recordGenerator.getRecordReader();

http://git-wip-us.apache.org/repos/asf/drill/blob/d0464ab9/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/StaticDrillTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/StaticDrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/StaticDrillTable.java
index 0a9b9b3..809c072 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/StaticDrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/StaticDrillTable.java
@@ -19,9 +19,11 @@ package org.apache.drill.exec.store.sys;
 
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.Schema.TableType;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.store.RecordDataType;
 import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.util.ImpersonationUtil;
 
 /**
  * A {@link org.apache.drill.exec.planner.logical.DrillTable} with a defined schema
@@ -32,8 +34,8 @@ public class StaticDrillTable extends DrillTable {
 
   private final RecordDataType dataType;
 
-  public StaticDrillTable(String storageEngineName, StoragePlugin plugin, Object selection, RecordDataType dataType) {
-    super(storageEngineName, plugin, selection);
+  public StaticDrillTable(String storageEngineName, StoragePlugin plugin, TableType tableType, Object selection, RecordDataType dataType) {
+    super(storageEngineName, plugin, tableType, ImpersonationUtil.getProcessUserName(), selection);
     this.dataType = dataType;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/d0464ab9/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTablePlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTablePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTablePlugin.java
index 4fb0475..948aa0f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTablePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTablePlugin.java
@@ -21,10 +21,6 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Set;
 
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.expression.SchemaPath;
@@ -37,6 +33,11 @@ import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.store.pojo.PojoDataType;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
 /**
  * A "storage" plugin for system tables.
  */
@@ -103,8 +104,8 @@ public class SystemTablePlugin extends AbstractStoragePlugin {
     public DrillTable getTable(String name) {
       for (SystemTable table : SystemTable.values()) {
         if (table.getTableName().equalsIgnoreCase(name)) {
-          return new StaticDrillTable(SystemTablePlugin.this.name, SystemTablePlugin.this, table,
-            new PojoDataType(table.getPojoClass()));
+          return new StaticDrillTable(SystemTablePlugin.this.name, SystemTablePlugin.this, TableType.SYSTEM_TABLE,
+            table, new PojoDataType(table.getPojoClass()));
         }
       }
       return null;

http://git-wip-us.apache.org/repos/asf/drill/blob/d0464ab9/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
index 5ea8bfc..d433d12 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
@@ -18,9 +18,11 @@
 package org.apache.drill.exec.work.metadata;
 
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_NAME;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_COLUMN_NAME;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SCHS_COL_SCHEMA_NAME;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SHRD_COL_TABLE_NAME;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SHRD_COL_TABLE_SCHEMA;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.TBLS_COL_TABLE_TYPE;
 import static org.apache.drill.exec.store.ischema.InfoSchemaTableType.CATALOGS;
 import static org.apache.drill.exec.store.ischema.InfoSchemaTableType.COLUMNS;
 import static org.apache.drill.exec.store.ischema.InfoSchemaTableType.SCHEMATA;
@@ -38,8 +40,8 @@ import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType;
 import org.apache.drill.exec.proto.UserProtos.CatalogMetadata;
 import org.apache.drill.exec.proto.UserProtos.ColumnMetadata;
-import org.apache.drill.exec.proto.UserProtos.GetCatalogsResp;
 import org.apache.drill.exec.proto.UserProtos.GetCatalogsReq;
+import org.apache.drill.exec.proto.UserProtos.GetCatalogsResp;
 import org.apache.drill.exec.proto.UserProtos.GetColumnsReq;
 import org.apache.drill.exec.proto.UserProtos.GetColumnsResp;
 import org.apache.drill.exec.proto.UserProtos.GetSchemasReq;
@@ -58,7 +60,6 @@ import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.store.SchemaConfig.SchemaConfigInfoProvider;
 import org.apache.drill.exec.store.SchemaTreeProvider;
-import org.apache.drill.exec.store.ischema.InfoSchemaConstants;
 import org.apache.drill.exec.store.ischema.InfoSchemaFilter;
 import org.apache.drill.exec.store.ischema.InfoSchemaFilter.ConstantExprNode;
 import org.apache.drill.exec.store.ischema.InfoSchemaFilter.ExprNode;
@@ -82,6 +83,7 @@ import com.google.common.collect.Ordering;
 public class MetadataProvider {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MetadataProvider.class);
 
+  private static final String IN_FUNCTION = "in";
   private static final String LIKE_FUNCTION = "like";
   private static final String AND_FUNCTION = "booleanand";
   private static final String OR_FUNCTION = "booleanor";
@@ -171,11 +173,11 @@ public class MetadataProvider {
     protected Response runInternal(final UserSession session, final SchemaTreeProvider schemaProvider) {
       final GetCatalogsResp.Builder respBuilder = GetCatalogsResp.newBuilder();
       final InfoSchemaFilter filter = createInfoSchemaFilter(
-          req.hasCatalogNameFilter() ? req.getCatalogNameFilter() : null, null, null, null);
+          req.hasCatalogNameFilter() ? req.getCatalogNameFilter() : null, null, null, null, null);
 
       try {
         final PojoRecordReader<Catalog> records =
-            (PojoRecordReader<Catalog>) getPojoRecordReader(CATALOGS, filter, schemaProvider, session);
+            getPojoRecordReader(CATALOGS, filter, schemaProvider, session);
 
         List<CatalogMetadata> metadata = new ArrayList<>();
         for(Catalog c : records) {
@@ -226,11 +228,11 @@ public class MetadataProvider {
 
       final InfoSchemaFilter filter = createInfoSchemaFilter(
           req.hasCatalogNameFilter() ? req.getCatalogNameFilter() : null,
-          req.hasSchameNameFilter() ? req.getSchameNameFilter() : null,
-          null, null);
+          req.hasSchemaNameFilter() ? req.getSchemaNameFilter() : null,
+          null, null, null);
 
       try {
-        final PojoRecordReader<Schema> records = (PojoRecordReader<Schema>)
+        final PojoRecordReader<Schema> records =
             getPojoRecordReader(SCHEMATA, filter, schemaProvider, session);
 
         List<SchemaMetadata> metadata = new ArrayList<>();
@@ -284,13 +286,14 @@ public class MetadataProvider {
 
       final InfoSchemaFilter filter = createInfoSchemaFilter(
           req.hasCatalogNameFilter() ? req.getCatalogNameFilter() : null,
-          req.hasSchameNameFilter() ? req.getSchameNameFilter() : null,
+          req.hasSchemaNameFilter() ? req.getSchemaNameFilter() : null,
           req.hasTableNameFilter() ? req.getTableNameFilter() : null,
+          req.getTableTypeFilterCount() != 0 ? req.getTableTypeFilterList() : null,
           null);
 
       try {
         final PojoRecordReader<Table> records =
-            (PojoRecordReader<Table>)getPojoRecordReader(TABLES, filter, schemaProvider, session);
+            getPojoRecordReader(TABLES, filter, schemaProvider, session);
 
         List<TableMetadata> metadata = new ArrayList<>();
         for(Table t : records) {
@@ -344,14 +347,14 @@ public class MetadataProvider {
 
       final InfoSchemaFilter filter = createInfoSchemaFilter(
           req.hasCatalogNameFilter() ? req.getCatalogNameFilter() : null,
-          req.hasSchameNameFilter() ? req.getSchameNameFilter() : null,
+          req.hasSchemaNameFilter() ? req.getSchemaNameFilter() : null,
           req.hasTableNameFilter() ? req.getTableNameFilter() : null,
-          req.hasColumnNameFilter() ? req.getColumnNameFilter() : null
+          null, req.hasColumnNameFilter() ? req.getColumnNameFilter() : null
       );
 
       try {
         final PojoRecordReader<Column> records =
-            (PojoRecordReader<Column>)getPojoRecordReader(COLUMNS, filter, schemaProvider, session);
+            getPojoRecordReader(COLUMNS, filter, schemaProvider, session);
 
         List<ColumnMetadata> metadata = new ArrayList<>();
         for(Column c : records) {
@@ -407,7 +410,7 @@ public class MetadataProvider {
 
         respBuilder.addAllColumns(metadata);
         respBuilder.setStatus(RequestStatus.OK);
-      } catch (Exception e) {
+      } catch (Throwable e) {
         respBuilder.setStatus(RequestStatus.FAILED);
         respBuilder.setError(createPBError("get columns", e));
       } finally {
@@ -421,11 +424,12 @@ public class MetadataProvider {
    * @param catalogNameFilter Optional filter on <code>catalog name</code>
    * @param schemaNameFilter Optional filter on <code>schema name</code>
    * @param tableNameFilter Optional filter on <code>table name</code>
+   * @param tableTypeFilter Optional filter on <code>table type</code>
    * @param columnNameFilter Optional filter on <code>column name</code>
    * @return
    */
   private static InfoSchemaFilter createInfoSchemaFilter(final LikeFilter catalogNameFilter,
-      final LikeFilter schemaNameFilter, final LikeFilter tableNameFilter, final LikeFilter columnNameFilter) {
+      final LikeFilter schemaNameFilter, final LikeFilter tableNameFilter, List<String> tableTypeFilter, final LikeFilter columnNameFilter) {
 
     FunctionExprNode exprNode = createLikeFunctionExprNode(CATS_COL_CATALOG_NAME,  catalogNameFilter);
 
@@ -444,7 +448,12 @@ public class MetadataProvider {
 
     exprNode = combineFunctions(AND_FUNCTION,
         exprNode,
-        createLikeFunctionExprNode(InfoSchemaConstants.COLS_COL_COLUMN_NAME, columnNameFilter)
+        createInFunctionExprNode(TBLS_COL_TABLE_TYPE, tableTypeFilter)
+        );
+
+    exprNode = combineFunctions(AND_FUNCTION,
+        exprNode,
+        createLikeFunctionExprNode(COLS_COL_COLUMN_NAME, columnNameFilter)
     );
 
     return exprNode != null ? new InfoSchemaFilter(exprNode) : null;
@@ -465,15 +474,35 @@ public class MetadataProvider {
         likeFilter.hasEscape() ?
             ImmutableList.of(
                 new FieldExprNode(fieldName),
-                new ConstantExprNode(likeFilter.getRegex()),
+                new ConstantExprNode(likeFilter.getPattern()),
                 new ConstantExprNode(likeFilter.getEscape())) :
             ImmutableList.of(
                 new FieldExprNode(fieldName),
-                new ConstantExprNode(likeFilter.getRegex()))
+                new ConstantExprNode(likeFilter.getPattern()))
     );
   }
 
   /**
+   * Helper method to create {@link FunctionExprNode} from {@code List<String>}.
+   * @param fieldName Name of the filed on which the like expression is applied.
+   * @param valuesFilter a list of values
+   * @return {@link FunctionExprNode} for given arguments. Null if the <code>valuesFilter</code> is null.
+   */
+  private static FunctionExprNode createInFunctionExprNode(String fieldName, List<String> valuesFilter) {
+    if (valuesFilter == null) {
+      return null;
+    }
+
+    ImmutableList.Builder<ExprNode> nodes = ImmutableList.builder();
+    nodes.add(new FieldExprNode(fieldName));
+    for(String type: valuesFilter) {
+      nodes.add(new ConstantExprNode(type));
+    }
+
+    return new FunctionExprNode(IN_FUNCTION, nodes.build());
+  }
+
+  /**
    * Helper method to combine two {@link FunctionExprNode}s with a given <code>functionName</code>. If one of them is
    * null, other one is returned as it is.
    */
@@ -498,7 +527,7 @@ public class MetadataProvider {
    * @param userSession
    * @return
    */
-  private static PojoRecordReader getPojoRecordReader(final InfoSchemaTableType tableType, final InfoSchemaFilter filter,
+  private static <S> PojoRecordReader<S> getPojoRecordReader(final InfoSchemaTableType tableType, final InfoSchemaFilter filter,
       final SchemaTreeProvider provider, final UserSession userSession) {
     final SchemaPlus rootSchema =
         provider.createRootSchema(userSession.getCredentials().getUserName(), newSchemaConfigInfoProvider(userSession));

http://git-wip-us.apache.org/repos/asf/drill/blob/d0464ab9/exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java b/exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java
index bd2cdfb..cd59dde 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java
@@ -21,8 +21,10 @@ import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.IS_CATALOG
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.IS_CATALOG_DESCR;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.IS_CATALOG_NAME;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.drill.BaseTestQuery;
@@ -36,7 +38,6 @@ import org.apache.drill.exec.proto.UserProtos.LikeFilter;
 import org.apache.drill.exec.proto.UserProtos.RequestStatus;
 import org.apache.drill.exec.proto.UserProtos.SchemaMetadata;
 import org.apache.drill.exec.proto.UserProtos.TableMetadata;
-
 import org.junit.Test;
 
 /**
@@ -65,7 +66,7 @@ public class TestMetadataProvider extends BaseTestQuery {
     // test("SELECT * FROM INFORMATION_SCHEMA.CATALOGS " +
     //    "WHERE CATALOG_NAME LIKE '%DRI%' ESCAPE '\\'"); // SQL equivalent
     GetCatalogsResp resp =
-        client.getCatalogs(LikeFilter.newBuilder().setRegex("%DRI%").setEscape("\\").build()).get();
+        client.getCatalogs(LikeFilter.newBuilder().setPattern("%DRI%").setEscape("\\").build()).get();
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<CatalogMetadata> catalogs = resp.getCatalogsList();
@@ -83,7 +84,7 @@ public class TestMetadataProvider extends BaseTestQuery {
     //     WHERE CATALOG_NAME LIKE '%DRIj\\\\hgjh%' ESCAPE '\\'"); // SQL equivalent
 
     GetCatalogsResp resp =
-        client.getCatalogs(LikeFilter.newBuilder().setRegex("%DRIj\\%hgjh%").setEscape("\\").build()).get();
+        client.getCatalogs(LikeFilter.newBuilder().setPattern("%DRIj\\%hgjh%").setEscape("\\").build()).get();
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<CatalogMetadata> catalogs = resp.getCatalogsList();
@@ -115,7 +116,7 @@ public class TestMetadataProvider extends BaseTestQuery {
   public void schemasWithSchemaNameFilter() throws Exception {
     // test("SELECT * FROM INFORMATION_SCHEMA.SCHEMATA WHERE SCHEMA_NAME LIKE '%y%'"); // SQL equivalent
 
-    GetSchemasResp resp = client.getSchemas(null, LikeFilter.newBuilder().setRegex("%y%").build()).get();
+    GetSchemasResp resp = client.getSchemas(null, LikeFilter.newBuilder().setPattern("%y%").build()).get();
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<SchemaMetadata> schemas = resp.getSchemasList();
@@ -131,8 +132,8 @@ public class TestMetadataProvider extends BaseTestQuery {
     //    "WHERE CATALOG_NAME LIKE '%RI%' AND SCHEMA_NAME LIKE '%y%'"); // SQL equivalent
 
     GetSchemasResp resp = client.getSchemas(
-        LikeFilter.newBuilder().setRegex("%RI%").build(),
-        LikeFilter.newBuilder().setRegex("%dfs_test%").build()).get();
+        LikeFilter.newBuilder().setPattern("%RI%").build(),
+        LikeFilter.newBuilder().setPattern("%dfs_test%").build()).get();
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<SchemaMetadata> schemas = resp.getSchemasList();
@@ -147,7 +148,41 @@ public class TestMetadataProvider extends BaseTestQuery {
   public void tables() throws Exception {
     // test("SELECT * FROM INFORMATION_SCHEMA.`TABLES`"); // SQL equivalent
 
-    GetTablesResp resp = client.getTables(null, null, null).get();
+    GetTablesResp resp = client.getTables(null, null, null, null).get();
+
+    assertEquals(RequestStatus.OK, resp.getStatus());
+    List<TableMetadata> tables = resp.getTablesList();
+    assertEquals(11, tables.size());
+
+    verifyTable("INFORMATION_SCHEMA", "CATALOGS", tables);
+    verifyTable("INFORMATION_SCHEMA", "COLUMNS", tables);
+    verifyTable("INFORMATION_SCHEMA", "SCHEMATA", tables);
+    verifyTable("INFORMATION_SCHEMA", "TABLES", tables);
+    verifyTable("INFORMATION_SCHEMA", "VIEWS", tables);
+    verifyTable("sys", "boot", tables);
+    verifyTable("sys", "drillbits", tables);
+    verifyTable("sys", "memory", tables);
+    verifyTable("sys", "options", tables);
+    verifyTable("sys", "threads", tables);
+    verifyTable("sys", "version", tables);
+  }
+
+  @Test
+  public void tablesWithTableFilter() throws Exception {
+    // test("SELECT * FROM INFORMATION_SCHEMA.`TABLES` WHERE TABLE_TYPE IN ('TABLE')"); // SQL equivalent
+
+    GetTablesResp resp = client.getTables(null, null, null, Arrays.asList("TABLE")).get();
+
+    assertEquals(RequestStatus.OK, resp.getStatus());
+    List<TableMetadata> tables = resp.getTablesList();
+    assertTrue(tables.isEmpty());
+  }
+
+  @Test
+  public void tablesWithSystemTableFilter() throws Exception {
+    // test("SELECT * FROM INFORMATION_SCHEMA.`TABLES` WHERE TABLE_TYPE IN ('SYSTEM_TABLE')"); // SQL equivalent
+
+    GetTablesResp resp = client.getTables(null, null, null, Arrays.asList("SYSTEM_TABLE")).get();
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<TableMetadata> tables = resp.getTablesList();
@@ -171,7 +206,8 @@ public class TestMetadataProvider extends BaseTestQuery {
     // test("SELECT * FROM INFORMATION_SCHEMA.`TABLES` WHERE TABLE_NAME LIKE '%o%'"); // SQL equivalent
 
     GetTablesResp resp = client.getTables(null, null,
-        LikeFilter.newBuilder().setRegex("%o%").build()).get();
+        LikeFilter.newBuilder().setPattern("%o%").build(),
+        null).get();
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<TableMetadata> tables = resp.getTablesList();
@@ -189,8 +225,9 @@ public class TestMetadataProvider extends BaseTestQuery {
     //    "WHERE TABLE_SCHEMA LIKE '%N\\_S%' ESCAPE '\\' AND TABLE_NAME LIKE '%o%'"); // SQL equivalent
 
     GetTablesResp resp = client.getTables(null,
-        LikeFilter.newBuilder().setRegex("%N\\_S%").setEscape("\\").build(),
-        LikeFilter.newBuilder().setRegex("%o%").build()).get();
+        LikeFilter.newBuilder().setPattern("%N\\_S%").setEscape("\\").build(),
+        LikeFilter.newBuilder().setPattern("%o%").build(),
+        null).get();
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<TableMetadata> tables = resp.getTablesList();
@@ -214,7 +251,7 @@ public class TestMetadataProvider extends BaseTestQuery {
     // test("SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE COLUMN_NAME LIKE '%\\_p%' ESCAPE '\\'"); // SQL equivalent
 
     GetColumnsResp resp = client.getColumns(null, null, null,
-        LikeFilter.newBuilder().setRegex("%\\_p%").setEscape("\\").build()).get();
+        LikeFilter.newBuilder().setPattern("%\\_p%").setEscape("\\").build()).get();
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<ColumnMetadata> columns = resp.getColumnsList();
@@ -233,8 +270,8 @@ public class TestMetadataProvider extends BaseTestQuery {
     //     WHERE TABLE_NAME LIKE '%bits' AND COLUMN_NAME LIKE '%\\_p%' ESCAPE '\\'"); // SQL equivalent
 
     GetColumnsResp resp = client.getColumns(null, null,
-        LikeFilter.newBuilder().setRegex("%bits").build(),
-        LikeFilter.newBuilder().setRegex("%\\_p%").setEscape("\\").build()).get();
+        LikeFilter.newBuilder().setPattern("%bits").build(),
+        LikeFilter.newBuilder().setPattern("%\\_p%").setEscape("\\").build()).get();
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<ColumnMetadata> columns = resp.getColumnsList();
@@ -252,10 +289,10 @@ public class TestMetadataProvider extends BaseTestQuery {
     //    "TABLE_NAME LIKE '%bits' AND COLUMN_NAME LIKE '%\\_p%' ESCAPE '\\'"); // SQL equivalent
 
     GetColumnsResp resp = client.getColumns(
-        LikeFilter.newBuilder().setRegex("%ILL").build(),
-        LikeFilter.newBuilder().setRegex("sys").build(),
-        LikeFilter.newBuilder().setRegex("%bits").build(),
-        LikeFilter.newBuilder().setRegex("%\\_p%").setEscape("\\").build()).get();
+        LikeFilter.newBuilder().setPattern("%ILL").build(),
+        LikeFilter.newBuilder().setPattern("sys").build(),
+        LikeFilter.newBuilder().setPattern("%bits").build(),
+        LikeFilter.newBuilder().setPattern("%\\_p%").setEscape("\\").build()).get();
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<ColumnMetadata> columns = resp.getColumnsList();

http://git-wip-us.apache.org/repos/asf/drill/blob/d0464ab9/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcMetadata.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcMetadata.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcMetadata.java
index d5c9c71..6d766bd 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcMetadata.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcMetadata.java
@@ -74,7 +74,7 @@ public class TestJdbcMetadata extends JdbcTestActionBase {
   public void tablesWithConditions() throws Exception{
     this.testAction(new JdbcAction(){
       public ResultSet getResult(Connection c) throws SQLException {
-        return c.getMetaData().getTables("DRILL", "sys", "opt%", new String[]{"TABLE", "VIEW"});
+        return c.getMetaData().getTables("DRILL", "sys", "opt%", new String[]{"SYSTEM_TABLE", "SYSTEM_VIEW"});
       }
     }, 1);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/d0464ab9/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java
index 833869b..891b1c9 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java
@@ -938,8 +938,8 @@ public final class SchemaUserProtos
         {
             public void writeTo(com.dyuproject.protostuff.Output output, org.apache.drill.exec.proto.UserProtos.LikeFilter message) throws java.io.IOException
             {
-                if(message.hasRegex())
-                    output.writeString(1, message.getRegex(), false);
+                if(message.hasPattern())
+                    output.writeString(1, message.getPattern(), false);
                 if(message.hasEscape())
                     output.writeString(2, message.getEscape(), false);
             }
@@ -982,7 +982,7 @@ public final class SchemaUserProtos
                         case 0:
                             return;
                         case 1:
-                            builder.setRegex(input.readString());
+                            builder.setPattern(input.readString());
                             break;
                         case 2:
                             builder.setEscape(input.readString());
@@ -1027,7 +1027,7 @@ public final class SchemaUserProtos
         {
             switch(number)
             {
-                case 1: return "regex";
+                case 1: return "pattern";
                 case 2: return "escape";
                 default: return null;
             }
@@ -1040,7 +1040,7 @@ public final class SchemaUserProtos
         private static final java.util.HashMap<java.lang.String,java.lang.Integer> fieldMap = new java.util.HashMap<java.lang.String,java.lang.Integer>();
         static
         {
-            fieldMap.put("regex", 1);
+            fieldMap.put("pattern", 1);
             fieldMap.put("escape", 2);
         }
     }
@@ -1426,8 +1426,8 @@ public final class SchemaUserProtos
                 if(message.hasCatalogNameFilter())
                     output.writeObject(1, message.getCatalogNameFilter(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.WRITE, false);
 
-                if(message.hasSchameNameFilter())
-                    output.writeObject(2, message.getSchameNameFilter(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.WRITE, false);
+                if(message.hasSchemaNameFilter())
+                    output.writeObject(2, message.getSchemaNameFilter(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.WRITE, false);
 
             }
             public boolean isInitialized(org.apache.drill.exec.proto.UserProtos.GetSchemasReq message)
@@ -1473,7 +1473,7 @@ public final class SchemaUserProtos
 
                             break;
                         case 2:
-                            builder.setSchameNameFilter(input.mergeObject(org.apache.drill.exec.proto.UserProtos.LikeFilter.newBuilder(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.MERGE));
+                            builder.setSchemaNameFilter(input.mergeObject(org.apache.drill.exec.proto.UserProtos.LikeFilter.newBuilder(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.MERGE));
 
                             break;
                         default:
@@ -1517,7 +1517,7 @@ public final class SchemaUserProtos
             switch(number)
             {
                 case 1: return "catalogNameFilter";
-                case 2: return "schameNameFilter";
+                case 2: return "schemaNameFilter";
                 default: return null;
             }
         }
@@ -1530,7 +1530,7 @@ public final class SchemaUserProtos
         static
         {
             fieldMap.put("catalogNameFilter", 1);
-            fieldMap.put("schameNameFilter", 2);
+            fieldMap.put("schemaNameFilter", 2);
         }
     }
 
@@ -1816,12 +1816,14 @@ public final class SchemaUserProtos
                 if(message.hasCatalogNameFilter())
                     output.writeObject(1, message.getCatalogNameFilter(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.WRITE, false);
 
-                if(message.hasSchameNameFilter())
-                    output.writeObject(2, message.getSchameNameFilter(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.WRITE, false);
+                if(message.hasSchemaNameFilter())
+                    output.writeObject(2, message.getSchemaNameFilter(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.WRITE, false);
 
                 if(message.hasTableNameFilter())
                     output.writeObject(3, message.getTableNameFilter(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.WRITE, false);
 
+                for(String tableTypeFilter : message.getTableTypeFilterList())
+                    output.writeString(4, tableTypeFilter, true);
             }
             public boolean isInitialized(org.apache.drill.exec.proto.UserProtos.GetTablesReq message)
             {
@@ -1866,13 +1868,16 @@ public final class SchemaUserProtos
 
                             break;
                         case 2:
-                            builder.setSchameNameFilter(input.mergeObject(org.apache.drill.exec.proto.UserProtos.LikeFilter.newBuilder(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.MERGE));
+                            builder.setSchemaNameFilter(input.mergeObject(org.apache.drill.exec.proto.UserProtos.LikeFilter.newBuilder(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.MERGE));
 
                             break;
                         case 3:
                             builder.setTableNameFilter(input.mergeObject(org.apache.drill.exec.proto.UserProtos.LikeFilter.newBuilder(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.MERGE));
 
                             break;
+                        case 4:
+                            builder.addTableTypeFilter(input.readString());
+                            break;
                         default:
                             input.handleUnknownField(number, this);
                     }
@@ -1914,8 +1919,9 @@ public final class SchemaUserProtos
             switch(number)
             {
                 case 1: return "catalogNameFilter";
-                case 2: return "schameNameFilter";
+                case 2: return "schemaNameFilter";
                 case 3: return "tableNameFilter";
+                case 4: return "tableTypeFilter";
                 default: return null;
             }
         }
@@ -1928,8 +1934,9 @@ public final class SchemaUserProtos
         static
         {
             fieldMap.put("catalogNameFilter", 1);
-            fieldMap.put("schameNameFilter", 2);
+            fieldMap.put("schemaNameFilter", 2);
             fieldMap.put("tableNameFilter", 3);
+            fieldMap.put("tableTypeFilter", 4);
         }
     }
 
@@ -2208,8 +2215,8 @@ public final class SchemaUserProtos
                 if(message.hasCatalogNameFilter())
                     output.writeObject(1, message.getCatalogNameFilter(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.WRITE, false);
 
-                if(message.hasSchameNameFilter())
-                    output.writeObject(2, message.getSchameNameFilter(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.WRITE, false);
+                if(message.hasSchemaNameFilter())
+                    output.writeObject(2, message.getSchemaNameFilter(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.WRITE, false);
 
                 if(message.hasTableNameFilter())
                     output.writeObject(3, message.getTableNameFilter(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.WRITE, false);
@@ -2261,7 +2268,7 @@ public final class SchemaUserProtos
 
                             break;
                         case 2:
-                            builder.setSchameNameFilter(input.mergeObject(org.apache.drill.exec.proto.UserProtos.LikeFilter.newBuilder(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.MERGE));
+                            builder.setSchemaNameFilter(input.mergeObject(org.apache.drill.exec.proto.UserProtos.LikeFilter.newBuilder(), org.apache.drill.exec.proto.SchemaUserProtos.LikeFilter.MERGE));
 
                             break;
                         case 3:
@@ -2313,7 +2320,7 @@ public final class SchemaUserProtos
             switch(number)
             {
                 case 1: return "catalogNameFilter";
-                case 2: return "schameNameFilter";
+                case 2: return "schemaNameFilter";
                 case 3: return "tableNameFilter";
                 case 4: return "columnNameFilter";
                 default: return null;
@@ -2328,7 +2335,7 @@ public final class SchemaUserProtos
         static
         {
             fieldMap.put("catalogNameFilter", 1);
-            fieldMap.put("schameNameFilter", 2);
+            fieldMap.put("schemaNameFilter", 2);
             fieldMap.put("tableNameFilter", 3);
             fieldMap.put("columnNameFilter", 4);
         }