You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2015/04/25 01:22:45 UTC

[10/11] drill git commit: DRILL-2829: Info. schema hygiene (for upcoming fixes).

DRILL-2829: Info. schema hygiene (for upcoming fixes).

- Clarified column constant names.  Also grouped and ordered them.
- Added/applied constant for catalog name "DRILL".
- Fixed "implements ...Constants" old-Java hack (to static imports).
- Purged some unused imports.
- A little documentation, editing, TODOs.


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

Branch: refs/heads/master
Commit: 1aad02c51a762364c2964cf7b60c141419a02bcc
Parents: 9e4c016
Author: dbarclay <db...@maprtech.com>
Authored: Fri Apr 17 11:36:46 2015 -0700
Committer: Parth Chandra <pc...@maprtech.com>
Committed: Fri Apr 24 16:21:55 2015 -0700

----------------------------------------------------------------------
 .../sql/handlers/DescribeTableHandler.java      | 31 +++++-----
 .../sql/handlers/ShowSchemasHandler.java        | 11 ++--
 .../planner/sql/handlers/ShowTablesHandler.java | 12 ++--
 .../apache/drill/exec/store/AbstractSchema.java |  9 +++
 .../exec/store/ischema/InfoSchemaConstants.java | 65 +++++++++++++-------
 .../store/ischema/InfoSchemaFilterBuilder.java  | 12 ++--
 .../store/ischema/InfoSchemaStoragePlugin.java  |  6 +-
 .../exec/store/ischema/InfoSchemaTable.java     | 62 ++++++++++---------
 .../exec/store/ischema/RecordGenerator.java     | 47 +++++++++-----
 .../drill/exec/store/ischema/Records.java       |  3 +-
 .../drill/exec/store/ischema/SelectedTable.java | 12 +++-
 .../drill/exec/store/sys/SystemTable.java       | 10 +--
 12 files changed, 177 insertions(+), 103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/1aad02c5/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java
index 15721d5..c76914b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java
@@ -26,13 +26,11 @@ import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.tools.Planner;
 import org.apache.calcite.tools.RelConversionException;
 
-import org.apache.drill.exec.ops.QueryContext;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.*;
 import org.apache.drill.exec.planner.sql.parser.DrillParserUtil;
 import org.apache.drill.exec.planner.sql.parser.SqlDescribeTable;
 import org.apache.drill.exec.store.AbstractSchema;
-import org.apache.drill.exec.store.ischema.InfoSchemaConstants;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
-import org.apache.calcite.plan.hep.HepPlanner;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
@@ -44,7 +42,7 @@ import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
-public class DescribeTableHandler extends DefaultSqlHandler implements InfoSchemaConstants {
+public class DescribeTableHandler extends DefaultSqlHandler {
 
   public DescribeTableHandler(SqlHandlerConfig config) { super(config); }
 
@@ -54,9 +52,10 @@ public class DescribeTableHandler extends DefaultSqlHandler implements InfoSchem
     SqlDescribeTable node = unwrap(sqlNode, SqlDescribeTable.class);
 
     try {
-      List<SqlNode> selectList = ImmutableList.of((SqlNode) new SqlIdentifier(COL_COLUMN_NAME, SqlParserPos.ZERO),
-          new SqlIdentifier(COL_DATA_TYPE, SqlParserPos.ZERO),
-          new SqlIdentifier(COL_IS_NULLABLE, SqlParserPos.ZERO));
+      List<SqlNode> selectList =
+          ImmutableList.of((SqlNode) new SqlIdentifier(COLS_COL_COLUMN_NAME, SqlParserPos.ZERO),
+                                     new SqlIdentifier(COLS_COL_DATA_TYPE, SqlParserPos.ZERO),
+                                     new SqlIdentifier(COLS_COL_IS_NULLABLE, SqlParserPos.ZERO));
 
       SqlNode fromClause = new SqlIdentifier(
           ImmutableList.of(IS_SCHEMA_NAME, TAB_COLUMNS), null, SqlParserPos.ZERO, null);
@@ -75,14 +74,14 @@ public class DescribeTableHandler extends DefaultSqlHandler implements InfoSchem
         AbstractSchema drillSchema = getDrillSchema(schema);
 
         schemaCondition = DrillParserUtil.createCondition(
-            new SqlIdentifier(COL_TABLE_SCHEMA, SqlParserPos.ZERO),
+            new SqlIdentifier(SHRD_COL_TABLE_SCHEMA, SqlParserPos.ZERO),
             SqlStdOperatorTable.EQUALS,
             SqlLiteral.createCharString(drillSchema.getFullSchemaName(), CHARSET, SqlParserPos.ZERO)
         );
       }
 
       SqlNode where = DrillParserUtil.createCondition(
-          new SqlIdentifier(COL_TABLE_NAME, SqlParserPos.ZERO),
+          new SqlIdentifier(SHRD_COL_TABLE_NAME, SqlParserPos.ZERO),
           SqlStdOperatorTable.EQUALS,
           SqlLiteral.createCharString(tableName, CHARSET, SqlParserPos.ZERO));
 
@@ -90,12 +89,16 @@ public class DescribeTableHandler extends DefaultSqlHandler implements InfoSchem
 
       SqlNode columnFilter = null;
       if (node.getColumn() != null) {
-        columnFilter = DrillParserUtil.createCondition(new SqlIdentifier(COL_COLUMN_NAME, SqlParserPos.ZERO),
-            SqlStdOperatorTable.EQUALS,
-            SqlLiteral.createCharString(node.getColumn().toString(), CHARSET, SqlParserPos.ZERO));
+        columnFilter =
+            DrillParserUtil.createCondition(
+                new SqlIdentifier(COLS_COL_COLUMN_NAME, SqlParserPos.ZERO),
+                SqlStdOperatorTable.EQUALS,
+                SqlLiteral.createCharString(node.getColumn().toString(), CHARSET, SqlParserPos.ZERO));
       } else if (node.getColumnQualifier() != null) {
-        columnFilter = DrillParserUtil.createCondition(new SqlIdentifier(COL_COLUMN_NAME, SqlParserPos.ZERO),
-            SqlStdOperatorTable.LIKE, node.getColumnQualifier());
+        columnFilter =
+            DrillParserUtil.createCondition(
+                new SqlIdentifier(COLS_COL_COLUMN_NAME, SqlParserPos.ZERO),
+                SqlStdOperatorTable.LIKE, node.getColumnQualifier());
       }
 
       where = DrillParserUtil.createCondition(where, SqlStdOperatorTable.AND, columnFilter);

http://git-wip-us.apache.org/repos/asf/drill/blob/1aad02c5/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowSchemasHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowSchemasHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowSchemasHandler.java
index a037be3..d759d1e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowSchemasHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowSchemasHandler.java
@@ -24,7 +24,7 @@ import org.apache.calcite.tools.RelConversionException;
 
 import org.apache.drill.exec.planner.sql.parser.DrillParserUtil;
 import org.apache.drill.exec.planner.sql.parser.SqlShowSchemas;
-import org.apache.drill.exec.store.ischema.InfoSchemaConstants;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.*;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
@@ -35,7 +35,7 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 
 import com.google.common.collect.ImmutableList;
 
-public class ShowSchemasHandler extends DefaultSqlHandler implements InfoSchemaConstants {
+public class ShowSchemasHandler extends DefaultSqlHandler {
 
   public ShowSchemasHandler(SqlHandlerConfig config) { super(config); }
 
@@ -43,7 +43,8 @@ public class ShowSchemasHandler extends DefaultSqlHandler implements InfoSchemaC
   @Override
   public SqlNode rewrite(SqlNode sqlNode) throws RelConversionException, ForemanSetupException {
     SqlShowSchemas node = unwrap(sqlNode, SqlShowSchemas.class);
-    List<SqlNode> selectList = ImmutableList.of((SqlNode) new SqlIdentifier(COL_SCHEMA_NAME, SqlParserPos.ZERO));
+    List<SqlNode> selectList =
+        ImmutableList.of((SqlNode) new SqlIdentifier(SCHS_COL_SCHEMA_NAME, SqlParserPos.ZERO));
 
     SqlNode fromClause = new SqlIdentifier(
         ImmutableList.of(IS_SCHEMA_NAME, TAB_SCHEMATA), null, SqlParserPos.ZERO, null);
@@ -51,8 +52,8 @@ public class ShowSchemasHandler extends DefaultSqlHandler implements InfoSchemaC
     SqlNode where = null;
     final SqlNode likePattern = node.getLikePattern();
     if (likePattern != null) {
-      where = DrillParserUtil.createCondition(new SqlIdentifier(COL_SCHEMA_NAME, SqlParserPos.ZERO),
-          SqlStdOperatorTable.LIKE, likePattern);
+      where = DrillParserUtil.createCondition(new SqlIdentifier(SCHS_COL_SCHEMA_NAME, SqlParserPos.ZERO),
+                                              SqlStdOperatorTable.LIKE, likePattern);
     } else if (node.getWhereClause() != null) {
       where = node.getWhereClause();
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/1aad02c5/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowTablesHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowTablesHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowTablesHandler.java
index 4ac81e9..3d42f76 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowTablesHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowTablesHandler.java
@@ -28,7 +28,7 @@ import org.apache.calcite.tools.RelConversionException;
 import org.apache.drill.exec.planner.sql.parser.DrillParserUtil;
 import org.apache.drill.exec.planner.sql.parser.SqlShowTables;
 import org.apache.drill.exec.store.AbstractSchema;
-import org.apache.drill.exec.store.ischema.InfoSchemaConstants;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.*;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlLiteral;
@@ -41,7 +41,7 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
-public class ShowTablesHandler extends DefaultSqlHandler implements InfoSchemaConstants {
+public class ShowTablesHandler extends DefaultSqlHandler {
 
   public ShowTablesHandler(SqlHandlerConfig config) { super(config); }
 
@@ -54,8 +54,8 @@ public class ShowTablesHandler extends DefaultSqlHandler implements InfoSchemaCo
     SqlNode where;
 
     // create select columns
-    selectList.add(new SqlIdentifier(COL_TABLE_SCHEMA, SqlParserPos.ZERO));
-    selectList.add(new SqlIdentifier(COL_TABLE_NAME, SqlParserPos.ZERO));
+    selectList.add(new SqlIdentifier(SHRD_COL_TABLE_SCHEMA, SqlParserPos.ZERO));
+    selectList.add(new SqlIdentifier(SHRD_COL_TABLE_NAME, SqlParserPos.ZERO));
 
     fromClause = new SqlIdentifier(ImmutableList.of(IS_SCHEMA_NAME, TAB_TABLES), SqlParserPos.ZERO);
 
@@ -84,7 +84,7 @@ public class ShowTablesHandler extends DefaultSqlHandler implements InfoSchemaCo
     }
 
     where = DrillParserUtil.createCondition(
-        new SqlIdentifier(COL_TABLE_SCHEMA, SqlParserPos.ZERO),
+        new SqlIdentifier(SHRD_COL_TABLE_SCHEMA, SqlParserPos.ZERO),
         SqlStdOperatorTable.EQUALS,
         SqlLiteral.createCharString(tableSchema, CHARSET, SqlParserPos.ZERO));
 
@@ -92,7 +92,7 @@ public class ShowTablesHandler extends DefaultSqlHandler implements InfoSchemaCo
     final SqlNode likePattern = node.getLikePattern();
     if (likePattern != null) {
       filter = DrillParserUtil.createCondition(
-          new SqlIdentifier(COL_TABLE_NAME, SqlParserPos.ZERO),
+          new SqlIdentifier(SHRD_COL_TABLE_NAME, SqlParserPos.ZERO),
           SqlStdOperatorTable.LIKE,
           likePattern);
     } else if (node.getWhereClause() != null) {

http://git-wip-us.apache.org/repos/asf/drill/blob/1aad02c5/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
index 2d6ac4f..33ddea5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
@@ -92,6 +92,15 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer
     throw new UnsupportedOperationException("New tables are not allowed in this schema");
   }
 
+  /**
+   * Reports whether to show items from this schema in INFORMATION_SCHEMA
+   * tables.
+   * (Controls ... TODO:  Doc.:  Mention what this typically controls or
+   * affects.)
+   * <p>
+   *   This base implementation returns {@code true}.
+   * </p>
+   */
   public boolean showInInformationSchema() {
     return true;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/1aad02c5/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConstants.java
index 8350d89..1c29235 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConstants.java
@@ -17,35 +17,58 @@
  */
 package org.apache.drill.exec.store.ischema;
 
-public interface InfoSchemaConstants {
+public final class InfoSchemaConstants {
+  /** Prevents instantiation. */
+  private InfoSchemaConstants() {
+  }
+
+  /** Name of catalog containing information schema. */
+  public static final String IS_CATALOG_NAME = "DRILL";
+
+  /** Name of information schema. */
   public static final String IS_SCHEMA_NAME = "INFORMATION_SCHEMA";
 
+  // TODO:  Resolve how to not have two different place defining table names:
+  // NOTE: These string values have to match the identifiers for SelectedTable's
+  // enumerators.
+  // Information schema's tables' names:
   public static final String TAB_CATALOGS = "CATALOGS";
   public static final String TAB_COLUMNS = "COLUMNS";
   public static final String TAB_SCHEMATA = "SCHEMATA";
   public static final String TAB_TABLES = "TABLES";
   public static final String TAB_VIEWS = "VIEWS";
 
+  // CATALOGS column names:
+  public static final String CATS_COL_CATALOG_CONNECT = "CATALOG_CONNECT";
+  public static final String CATS_COL_CATALOG_DESCRIPTION = "CATALOG_DESCRIPTION";
+  public static final String CATS_COL_CATALOG_NAME = "CATALOG_NAME";
+
+  // SCHEMATA column names:
+  public static final String SCHS_COL_CATALOG_NAME = "CATALOG_NAME";
+  public static final String SCHS_COL_SCHEMA_NAME = "SCHEMA_NAME";
+  public static final String SCHS_COL_SCHEMA_OWNER = "SCHEMA_OWNER";
+  public static final String SCHS_COL_TYPE = "TYPE";
+  public static final String SCHS_COL_IS_MUTABLE = "IS_MUTABLE";
+
+  // Common TABLES / VIEWS / COLUMNS columns names:
+  public static final String SHRD_COL_TABLE_CATALOG = "TABLE_CATALOG";
+  public static final String SHRD_COL_TABLE_SCHEMA = "TABLE_SCHEMA";
+  public static final String SHRD_COL_TABLE_NAME = "TABLE_NAME";
+
+  // Remaining TABLES column names:
+  public static final String TBLS_COL_TABLE_TYPE = "TABLE_TYPE";
+
+  // Remaining VIEWS column names:
+  public static final String VIEWS_COL_VIEW_DEFINITION = "VIEW_DEFINITION";
 
-  public static final String COL_CATALOG_CONNECT = "CATALOG_CONNECT";
-  public static final String COL_CATALOG_DESCRIPTION = "CATALOG_DESCRIPTION";
-  public static final String COL_CATALOG_NAME = "CATALOG_NAME";
-  public static final String COL_COLUMN_NAME = "COLUMN_NAME";
-  public static final String COL_CHARACTER_MAXIMUM_LENGTH = "CHARACTER_MAXIMUM_LENGTH";
-  public static final String COL_DATA_TYPE = "DATA_TYPE";
-  public static final String COL_IS_MUTABLE = "IS_MUTABLE";
-  public static final String COL_IS_NULLABLE = "IS_NULLABLE";
-  public static final String COL_NUMERIC_PRECISION = "NUMERIC_PRECISION";
-  public static final String COL_NUMERIC_PRECISION_RADIX = "NUMERIC_PRECISION_RADIX";
-  public static final String COL_NUMERIC_SCALE = "NUMERIC_SCALE";
-  public static final String COL_ORDINAL_POSITION = "ORDINAL_POSITION";
-  public static final String COL_SCHEMA_NAME = "SCHEMA_NAME";
-  public static final String COL_SCHEMA_OWNER = "SCHEMA_OWNER";
-  public static final String COL_TYPE = "TYPE";
-  public static final String COL_TABLE_NAME = "TABLE_NAME";
-  public static final String COL_TABLE_SCHEMA = "TABLE_SCHEMA";
-  public static final String COL_TABLE_CATALOG = "TABLE_CATALOG";
-  public static final String COL_TABLE_TYPE = "TABLE_TYPE";
-  public static final String COL_VIEW_DEFINITION = "VIEW_DEFINITION";
+  // Remaining COLUMNS column names:
+  public static final String COLS_COL_COLUMN_NAME = "COLUMN_NAME";
+  public static final String COLS_COL_ORDINAL_POSITION = "ORDINAL_POSITION";
+  public static final String COLS_COL_IS_NULLABLE = "IS_NULLABLE";
+  public static final String COLS_COL_DATA_TYPE = "DATA_TYPE";
+  public static final String COLS_COL_CHARACTER_MAXIMUM_LENGTH = "CHARACTER_MAXIMUM_LENGTH";
+  public static final String COLS_COL_NUMERIC_PRECISION_RADIX = "NUMERIC_PRECISION_RADIX";
+  public static final String COLS_COL_NUMERIC_SCALE = "NUMERIC_SCALE";
+  public static final String COLS_COL_NUMERIC_PRECISION = "NUMERIC_PRECISION";
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/1aad02c5/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilterBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilterBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilterBuilder.java
index ddbbe51..9d0bc06 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilterBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilterBuilder.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.store.ischema;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.*;
 import org.apache.drill.common.expression.BooleanOperator;
 import org.apache.drill.common.expression.CastExpression;
 import org.apache.drill.common.expression.FieldReference;
@@ -39,8 +40,7 @@ import java.util.List;
  * conditions involving columns "TABLE_NAME", "SCHEMA_NAME" and "TABLE_SCHEMA" and
  * functions EQUAL, NOT EQUAL, LIKE, OR and AND.
  */
-public class InfoSchemaFilterBuilder extends AbstractExprVisitor<ExprNode, Void, RuntimeException>
-    implements InfoSchemaConstants {
+public class InfoSchemaFilterBuilder extends AbstractExprVisitor<ExprNode, Void, RuntimeException> {
   private final LogicalExpression filter;
 
   private boolean isAllExpressionsConverted = true;
@@ -127,7 +127,9 @@ public class InfoSchemaFilterBuilder extends AbstractExprVisitor<ExprNode, Void,
     if (e.getInput() instanceof FieldReference) {
       FieldReference fieldRef = (FieldReference) e.getInput();
       String field = fieldRef.getAsUnescapedPath().toUpperCase();
-      if (field.equals(COL_SCHEMA_NAME) || field.equals(COL_TABLE_NAME) || field.equals(COL_TABLE_SCHEMA)) {
+      if (field.equals(SCHS_COL_SCHEMA_NAME)
+          || field.equals(SHRD_COL_TABLE_NAME)
+          || field.equals(SHRD_COL_TABLE_SCHEMA)) {
         return new FieldExprNode(field);
       }
     }
@@ -143,7 +145,9 @@ public class InfoSchemaFilterBuilder extends AbstractExprVisitor<ExprNode, Void,
   @Override
   public ExprNode visitSchemaPath(SchemaPath path, Void value) throws RuntimeException {
     String field = path.getAsUnescapedPath().toUpperCase();
-    if (field.equals(COL_SCHEMA_NAME) || field.equals(COL_TABLE_NAME) || field.equals(COL_TABLE_SCHEMA)) {
+    if (field.equals(SCHS_COL_SCHEMA_NAME)
+        || field.equals(SHRD_COL_TABLE_NAME)
+        || field.equals(SHRD_COL_TABLE_SCHEMA)) {
       return new FieldExprNode(field);
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/1aad02c5/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
index 0a9c32c..597d24c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
@@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.*;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
@@ -39,7 +40,7 @@ import com.google.common.collect.Maps;
 import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
 
-public class InfoSchemaStoragePlugin extends AbstractStoragePlugin implements InfoSchemaConstants {
+public class InfoSchemaStoragePlugin extends AbstractStoragePlugin {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InfoSchemaStoragePlugin.class);
 
   private final InfoSchemaConfig config;
@@ -75,6 +76,9 @@ public class InfoSchemaStoragePlugin extends AbstractStoragePlugin implements In
     parent.add(s.getName(), s);
   }
 
+  /**
+   * Representation of the INFORMATION_SCHEMA schema.
+   */
   private class ISchema extends AbstractSchema{
     private Map<String, InfoSchemaDrillTable> tables;
     public ISchema(SchemaPlus parent, InfoSchemaStoragePlugin plugin){

http://git-wip-us.apache.org/repos/asf/drill/blob/1aad02c5/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 0e9ca53..0f8b8a0 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
@@ -19,6 +19,7 @@ package org.apache.drill.exec.store.ischema;
 
 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;
@@ -29,8 +30,11 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
-/** Base class of tables in INFORMATION_SCHEMA. Defines the table (fields and types) */
-public abstract class InfoSchemaTable implements InfoSchemaConstants{
+/**
+ * Base class for tables in INFORMATION_SCHEMA.  Defines the table (fields and
+ * types).
+ */
+public abstract class InfoSchemaTable {
 
   public static class Field {
     public String name;
@@ -86,9 +90,9 @@ public abstract class InfoSchemaTable implements InfoSchemaConstants{
   /** Layout for the CATALOGS table. */
   static public class Catalogs extends InfoSchemaTable {
     private static final List<Field> fields = ImmutableList.of(
-        Field.create(COL_CATALOG_NAME, VARCHAR),
-        Field.create(COL_CATALOG_DESCRIPTION, VARCHAR),
-        Field.create(COL_CATALOG_CONNECT, VARCHAR));
+        Field.create(CATS_COL_CATALOG_NAME, VARCHAR),
+        Field.create(CATS_COL_CATALOG_DESCRIPTION, VARCHAR),
+        Field.create(CATS_COL_CATALOG_CONNECT, VARCHAR));
 
     Catalogs() {
       super(TAB_CATALOGS, fields);
@@ -103,11 +107,11 @@ public abstract class InfoSchemaTable implements InfoSchemaConstants{
   /** Layout for the SCHEMATA table. */
   public static class Schemata extends InfoSchemaTable {
     private static final List<Field> fields = ImmutableList.of(
-        Field.create(COL_CATALOG_NAME, VARCHAR),
-        Field.create(COL_SCHEMA_NAME, VARCHAR),
-        Field.create(COL_SCHEMA_OWNER, VARCHAR),
-        Field.create(COL_TYPE, VARCHAR),
-        Field.create(COL_IS_MUTABLE, VARCHAR));
+        Field.create(SCHS_COL_CATALOG_NAME, VARCHAR),
+        Field.create(SCHS_COL_SCHEMA_NAME, VARCHAR),
+        Field.create(SCHS_COL_SCHEMA_OWNER, VARCHAR),
+        Field.create(SCHS_COL_TYPE, VARCHAR),
+        Field.create(SCHS_COL_IS_MUTABLE, VARCHAR));
 
     public Schemata() {
       super(TAB_SCHEMATA, fields);
@@ -122,10 +126,10 @@ public abstract class InfoSchemaTable implements InfoSchemaConstants{
   /** Layout for the TABLES table. */
   public static class Tables extends InfoSchemaTable {
     private static final List<Field> fields = ImmutableList.of(
-        Field.create(COL_TABLE_CATALOG, VARCHAR),
-        Field.create(COL_TABLE_SCHEMA, VARCHAR),
-        Field.create(COL_TABLE_NAME, VARCHAR),
-        Field.create(COL_TABLE_TYPE, VARCHAR));
+        Field.create(SHRD_COL_TABLE_CATALOG, VARCHAR),
+        Field.create(SHRD_COL_TABLE_SCHEMA, VARCHAR),
+        Field.create(SHRD_COL_TABLE_NAME, VARCHAR),
+        Field.create(TBLS_COL_TABLE_TYPE, VARCHAR));
 
     public Tables() {
       super(TAB_TABLES, fields);
@@ -140,10 +144,10 @@ public abstract class InfoSchemaTable implements InfoSchemaConstants{
   /** Layout for the VIEWS table. */
   static public class Views extends InfoSchemaTable {
     private static final List<Field> fields = ImmutableList.of(
-        Field.create(COL_TABLE_CATALOG, VARCHAR),
-        Field.create(COL_TABLE_SCHEMA, VARCHAR),
-        Field.create(COL_TABLE_NAME, VARCHAR),
-        Field.create(COL_VIEW_DEFINITION, VARCHAR));
+        Field.create(SHRD_COL_TABLE_CATALOG, VARCHAR),
+        Field.create(SHRD_COL_TABLE_SCHEMA, VARCHAR),
+        Field.create(SHRD_COL_TABLE_NAME, VARCHAR),
+        Field.create(VIEWS_COL_VIEW_DEFINITION, VARCHAR));
 
     public Views() {
       super(TAB_VIEWS, fields);
@@ -158,17 +162,17 @@ public abstract class InfoSchemaTable implements InfoSchemaConstants{
   /** Layout for the COLUMNS table. */
   public static class Columns extends InfoSchemaTable {
     private static final List<Field> fields = ImmutableList.of(
-        Field.create(COL_TABLE_CATALOG, VARCHAR),
-        Field.create(COL_TABLE_SCHEMA, VARCHAR),
-        Field.create(COL_TABLE_NAME, VARCHAR),
-        Field.create(COL_COLUMN_NAME, VARCHAR),
-        Field.create(COL_ORDINAL_POSITION, INT),
-        Field.create(COL_IS_NULLABLE, VARCHAR),
-        Field.create(COL_DATA_TYPE, VARCHAR),
-        Field.create(COL_CHARACTER_MAXIMUM_LENGTH, INT),
-        Field.create(COL_NUMERIC_PRECISION_RADIX, INT),
-        Field.create(COL_NUMERIC_SCALE, INT),
-        Field.create(COL_NUMERIC_PRECISION, INT));
+        Field.create(SHRD_COL_TABLE_CATALOG, VARCHAR),
+        Field.create(SHRD_COL_TABLE_SCHEMA, VARCHAR),
+        Field.create(SHRD_COL_TABLE_NAME, VARCHAR),
+        Field.create(COLS_COL_COLUMN_NAME, VARCHAR),
+        Field.create(COLS_COL_ORDINAL_POSITION, INT),
+        Field.create(COLS_COL_IS_NULLABLE, VARCHAR),
+        Field.create(COLS_COL_DATA_TYPE, VARCHAR),
+        Field.create(COLS_COL_CHARACTER_MAXIMUM_LENGTH, INT),
+        Field.create(COLS_COL_NUMERIC_PRECISION_RADIX, INT),
+        Field.create(COLS_COL_NUMERIC_SCALE, INT),
+        Field.create(COLS_COL_NUMERIC_PRECISION, INT));
 
     public Columns() {
       super(TAB_COLUMNS, fields);

http://git-wip-us.apache.org/repos/asf/drill/blob/1aad02c5/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/RecordGenerator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/RecordGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/RecordGenerator.java
index 772b9e4..c2a56ca 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/RecordGenerator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/RecordGenerator.java
@@ -26,19 +26,23 @@ import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.*;
 import org.apache.drill.exec.planner.logical.DrillViewInfoProvider;
 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;
+
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
-/** Generates records for POJO RecordReader by scanning the given schema */
-public abstract class RecordGenerator implements InfoSchemaConstants {
+/**
+ * Generates records for POJO RecordReader by scanning the given schema.
+ */
+public abstract class RecordGenerator {
   protected InfoSchemaFilter filter;
 
   public void setInfoSchemaFilter(InfoSchemaFilter filter) {
@@ -69,21 +73,25 @@ public abstract class RecordGenerator implements InfoSchemaConstants {
         return false;
       }
 
-      Map<String, String> recordValues = ImmutableMap.of(COL_TABLE_SCHEMA, schemaName, COL_SCHEMA_NAME, schemaName);
+      Map<String, String> recordValues =
+          ImmutableMap.of(SHRD_COL_TABLE_SCHEMA, schemaName,
+                          SCHS_COL_SCHEMA_NAME, schemaName);
       if (filter != null && filter.evaluate(recordValues) == Result.FALSE) {
-        // If the filter evaluates to false then we don't need to visit the schema. For other two results (TRUE,
-        // INCONCLUSIVE) continue to visit the schema.
+        // If the filter evaluates to false then we don't need to visit the schema.
+        // For other two results (TRUE, INCONCLUSIVE) continue to visit the schema.
         return false;
       }
     } catch(ClassCastException e) {
-      // ignore and return true as this is not a drill schema
+      // ignore and return true as this is not a Drill schema
     }
     return true;
   }
 
   protected boolean shouldVisitTable(String schemaName, String tableName) {
-    Map<String, String> recordValues = ImmutableMap.of(
-        COL_TABLE_SCHEMA, schemaName, COL_SCHEMA_NAME, schemaName, COL_TABLE_NAME, tableName);
+    Map<String, String> recordValues =
+        ImmutableMap.of( SHRD_COL_TABLE_SCHEMA, schemaName,
+                         SCHS_COL_SCHEMA_NAME, schemaName,
+                         SHRD_COL_TABLE_NAME, tableName);
     if (filter != null && filter.evaluate(recordValues) == Result.FALSE) {
       return false;
     }
@@ -98,13 +106,13 @@ public abstract class RecordGenerator implements InfoSchemaConstants {
   }
 
   /**
-   * Recursively scan the schema, invoking the visitor as appropriate.
-   * @param schemaPath - the path to the current schema, so far,
-   * @param schema - the current schema.
+   * Recursively scans the given schema, invoking the visitor as appropriate.
+   * @param  schemaPath  the path to the given schema, so far
+   * @param  schema  the given schema
    */
   private void scanSchema(String schemaPath, SchemaPlus schema) {
 
-    // Recursively scan the subschema.
+    // Recursively scan any subschema.
     for (String name: schema.getSubSchemaNames()) {
       scanSchema(schemaPath +
           (schemaPath == "" ? "" : ".") + // If we have an empty schema path, then don't insert a leading dot.
@@ -138,7 +146,9 @@ public abstract class RecordGenerator implements InfoSchemaConstants {
   public static class Catalogs extends RecordGenerator {
     @Override
     public RecordReader getRecordReader() {
-      Records.Catalog catalogRecord = new Records.Catalog("DRILL", "The internal metadata used by Drill", "");
+      Records.Catalog catalogRecord =
+          new Records.Catalog(IS_CATALOG_NAME,
+                              "The internal metadata used by Drill", "");
       return new PojoRecordReader<>(Records.Catalog.class, ImmutableList.of(catalogRecord).iterator());
     }
   }
@@ -154,7 +164,8 @@ public abstract class RecordGenerator implements InfoSchemaConstants {
     @Override
     public boolean visitSchema(String schemaName, SchemaPlus schema) {
       AbstractSchema as = schema.unwrap(AbstractSchema.class);
-      records.add(new Records.Schema("DRILL", schemaName, "<owner>", as.getTypeName(), as.isMutable()));
+      records.add(new Records.Schema(IS_CATALOG_NAME, schemaName, "<owner>",
+                                     as.getTypeName(), as.isMutable()));
       return false;
     }
   }
@@ -169,7 +180,8 @@ public abstract class RecordGenerator implements InfoSchemaConstants {
 
     @Override
     public boolean visitTable(String schemaName, String tableName, Table table) {
-      records.add(new Records.Table("DRILL", schemaName, tableName, table.getJdbcTableType().toString()));
+      records.add(new Records.Table(IS_CATALOG_NAME, schemaName, tableName,
+                                    table.getJdbcTableType().toString()));
       return false;
     }
   }
@@ -185,7 +197,8 @@ public abstract class RecordGenerator implements InfoSchemaConstants {
     @Override
     public boolean visitTable(String schemaName, String tableName, Table table) {
       if (table.getJdbcTableType() == TableType.VIEW) {
-        records.add(new Records.View("DRILL", schemaName, tableName, ((DrillViewInfoProvider) table).getViewSql()));
+        records.add(new Records.View(IS_CATALOG_NAME, schemaName, tableName,
+                    ((DrillViewInfoProvider) table).getViewSql()));
       }
       return false;
     }
@@ -201,7 +214,7 @@ public abstract class RecordGenerator implements InfoSchemaConstants {
 
     @Override
     public boolean visitField(String schemaName, String tableName, RelDataTypeField field) {
-      records.add(new Records.Column("DRILL", schemaName, tableName, field));
+      records.add(new Records.Column(IS_CATALOG_NAME, schemaName, tableName, field));
       return false;
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/1aad02c5/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
index f14fa7e..39b4f3e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
@@ -66,7 +66,8 @@ public class Records {
       this.IS_NULLABLE = type.isNullable() ? "YES" : "NO";
 
       if (sqlType == SqlTypeName.ARRAY || sqlType == SqlTypeName.MAP || sqlType == SqlTypeName.ROW) {
-        // For complex types use the toString method to display the inside elements
+        // For complex types use SqlTypeName's toString method to display the
+        // inside elements.
         String typeString = type.toString();
 
         // RelDataType.toString prints "RecordType" for "STRUCT".

http://git-wip-us.apache.org/repos/asf/drill/blob/1aad02c5/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/SelectedTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/SelectedTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/SelectedTable.java
index 088736b..79e7fd2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/SelectedTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/SelectedTable.java
@@ -28,7 +28,13 @@ import org.apache.drill.exec.store.ischema.InfoSchemaTable.Views;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 
+/**
+ * The set of tables/views in INFORMATION_SCHEMA.
+ */
 public enum SelectedTable{
+  // TODO:  Resolve how to not have two different place defining table names:
+  // NOTE: These identifiers have to match the string values in
+  // InfoSchemaConstants.
   CATALOGS(new Catalogs()),
   SCHEMATA(new Schemata()),
   VIEWS(new Views()),
@@ -37,6 +43,10 @@ public enum SelectedTable{
 
   private final InfoSchemaTable tableDef;
 
+  /**
+   * ...
+   * @param  tableDef  the definition (columns and data generator) of the table
+   */
   private SelectedTable(InfoSchemaTable tableDef) {
     this.tableDef = tableDef;
   }
@@ -51,4 +61,4 @@ public enum SelectedTable{
   public RelDataType getRowType(RelDataTypeFactory typeFactory) {
     return tableDef.getRowType(typeFactory);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/1aad02c5/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTable.java
index 9a5aa65..bd23d81 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTable.java
@@ -30,10 +30,12 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import java.util.Iterator;
 
 /**
- * An enumeration of all system tables that Drill supports.
- * <p/>
- * OPTION, DRILLBITS and VERSION are local tables available on every Drillbit.
- * MEMORY and THREADS are distributed tables with one record on every Drillbit.
+ * An enumeration of all tables in Drill's system ("sys") schema.
+ * <p>
+ *   OPTION, DRILLBITS and VERSION are local tables available on every Drillbit.
+ *   MEMORY and THREADS are distributed tables with one record on every
+ *   Drillbit.
+ * </p>
  */
 public enum SystemTable {