You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2015/01/05 05:32:16 UTC

incubator-calcite git commit: [CALCITE-497] Support optional qualifier for column name references

Repository: incubator-calcite
Updated Branches:
  refs/heads/master 517465172 -> 175d0705c


[CALCITE-497] Support optional qualifier for column name references

Now, if a field is a record type, you can access its fields (and recursively their fields) without qualifiers if the field names are unique. So, this is useful for any document-oriented schema that is strongly typed, not just Phoenix/HBase.


Project: http://git-wip-us.apache.org/repos/asf/incubator-calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-calcite/commit/175d0705
Tree: http://git-wip-us.apache.org/repos/asf/incubator-calcite/tree/175d0705
Diff: http://git-wip-us.apache.org/repos/asf/incubator-calcite/diff/175d0705

Branch: refs/heads/master
Commit: 175d0705c308acc6f613ada70d2210908c6e8ccc
Parents: 5174651
Author: julianhyde <jh...@apache.org>
Authored: Sun Jan 4 18:46:01 2015 -0800
Committer: julianhyde <jh...@apache.org>
Committed: Sun Jan 4 18:46:01 2015 -0800

----------------------------------------------------------------------
 .../calcite/prepare/CalciteCatalogReader.java   |   6 +-
 .../apache/calcite/rel/type/RelDataType.java    |   8 +-
 .../calcite/rel/type/RelDataTypeImpl.java       |  51 +++++-
 .../java/org/apache/calcite/rex/RexBuilder.java |  17 +-
 .../apache/calcite/sql/type/SqlTypeUtil.java    |  13 ++
 .../calcite/sql/validate/SqlValidatorUtil.java  |  14 +-
 .../sql2rel/RelStructuredTypeFlattener.java     | 154 +++++++++----------
 .../java/org/apache/calcite/test/JdbcTest.java  |  56 +++----
 .../apache/calcite/test/MockCatalogReader.java  |   6 +-
 .../apache/calcite/test/SqlValidatorTest.java   |  14 ++
 .../enumerable/EnumerableCorrelateTest.java     |   4 +-
 .../org/apache/calcite/tools/PlannerTest.java   |  33 ++--
 core/src/test/resources/sql/misc.oq             |  16 +-
 .../calcite/adapter/splunk/SplunkTableScan.java |   2 +-
 14 files changed, 241 insertions(+), 153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
index 15cf374..263db18 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -73,6 +73,7 @@ public class CalciteCatalogReader implements Prepare.CatalogReader,
   final CalciteSchema rootSchema;
   final JavaTypeFactory typeFactory;
   private final List<String> defaultSchema;
+  private final boolean elideRecord = true;
   private final boolean caseSensitive;
 
   public CalciteCatalogReader(
@@ -201,7 +202,8 @@ public class CalciteCatalogReader implements Prepare.CatalogReader,
   }
 
   public RelDataTypeField field(RelDataType rowType, String alias) {
-    return SqlValidatorUtil.lookupField(caseSensitive, rowType, alias);
+    return SqlValidatorUtil.lookupField(caseSensitive, elideRecord, rowType,
+        alias);
   }
 
   public int fieldOrdinal(RelDataType rowType, String alias) {
@@ -220,7 +222,7 @@ public class CalciteCatalogReader implements Prepare.CatalogReader,
   public RelDataType createTypeFromProjection(final RelDataType type,
       final List<String> columnNameList) {
     return SqlValidatorUtil.createTypeFromProjection(type, columnNameList,
-        typeFactory, caseSensitive);
+        typeFactory, caseSensitive, elideRecord);
   }
 
   public void lookupOperatorOverloads(final SqlIdentifier opName,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java
index f378d48..0133470 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java
@@ -88,11 +88,13 @@ public interface RelDataType /*extends Type*/ {
    * <li>Hard-coding {@code false} is almost certainly wrong.</li>
    * </ul>
    *
-   * @param fieldName name of field to find
-   * @param caseSensitive Whether case-sensitive
+   * @param fieldName Name of field to find
+   * @param caseSensitive Whether match is case-sensitive
+   * @param elideRecord Whether to find fields nested within records
    * @return named field, or null if not found
    */
-  RelDataTypeField getField(String fieldName, boolean caseSensitive);
+  RelDataTypeField getField(String fieldName, boolean caseSensitive,
+      boolean elideRecord);
 
   /**
    * Queries whether this type allows null values.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java
index 3b13b9b..0c1335b 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java
@@ -27,6 +27,7 @@ import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 
 import java.io.Serializable;
 import java.nio.charset.Charset;
@@ -76,13 +77,28 @@ public abstract class RelDataTypeImpl
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement RelDataType
-  public RelDataTypeField getField(String fieldName, boolean caseSensitive) {
+  public RelDataTypeField getField(String fieldName, boolean caseSensitive,
+      boolean elideRecord) {
     for (RelDataTypeField field : fieldList) {
       if (Util.matches(caseSensitive, field.getName(), fieldName)) {
         return field;
       }
     }
+    if (elideRecord) {
+      final List<Slot> slots = Lists.newArrayList();
+      getFieldRecurse(slots, this, 0, fieldName, caseSensitive);
+    loop:
+      for (Slot slot : slots) {
+        switch (slot.count) {
+        case 0:
+          break; // no match at this depth; try deeper
+        case 1:
+          return slot.field;
+        default:
+          break loop; // duplicate fields at this depth; abandon search
+        }
+      }
+    }
     // Extra field
     if (fieldList.size() > 0) {
       final RelDataTypeField lastField = Iterables.getLast(fieldList);
@@ -94,6 +110,29 @@ public abstract class RelDataTypeImpl
     return null;
   }
 
+  private static void getFieldRecurse(List<Slot> slots, RelDataTypeImpl type,
+      int depth, String fieldName, boolean caseSensitive) {
+    while (slots.size() <= depth) {
+      slots.add(new Slot());
+    }
+    final Slot slot = slots.get(depth);
+    for (RelDataTypeField field : type.fieldList) {
+      if (Util.matches(caseSensitive, field.getName(), fieldName)) {
+        slot.count++;
+        slot.field = field;
+      }
+    }
+    // No point looking to depth + 1 if there is a hit at depth.
+    if (slot.count == 0) {
+      for (RelDataTypeField field : type.fieldList) {
+        if (field.getType().isStruct()) {
+          getFieldRecurse(slots, (RelDataTypeImpl) field.getType(), depth + 1,
+              fieldName, caseSensitive);
+        }
+      }
+    }
+  }
+
   // implement RelDataType
   public List<RelDataTypeField> getFieldList() {
     assert isStruct();
@@ -348,7 +387,13 @@ public abstract class RelDataTypeImpl
   public static RelDataTypeField extra(RelDataType rowType) {
     // Even in a case-insensitive connection, the name must be precisely
     // "_extra".
-    return rowType.getField("_extra", true);
+    return rowType.getField("_extra", true, false);
+  }
+
+  /** Work space for {@link RelDataTypeImpl#getFieldRecurse}. */
+  private static class Slot {
+    int count;
+    RelDataTypeField field;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
index 9dbac6c..81c6881 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
@@ -44,7 +44,9 @@ import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 import java.math.BigDecimal;
 import java.util.ArrayList;
@@ -67,6 +69,12 @@ public class RexBuilder {
    */
   public static final SqlSpecialOperator GET_OPERATOR =
       new SqlSpecialOperator("_get", SqlKind.OTHER_FUNCTION);
+  public static final Function<RelDataTypeField, RexInputRef> TO_INPUT_REF =
+      new Function<RelDataTypeField, RexInputRef>() {
+        public RexInputRef apply(RelDataTypeField input) {
+          return new RexInputRef(input.getIndex(), input.getType());
+        }
+      };
 
   //~ Instance fields --------------------------------------------------------
 
@@ -108,6 +116,12 @@ public class RexBuilder {
             SqlTypeName.NULL);
   }
 
+  /** Creates a list of {@link org.apache.calcite.rex.RexInputRef} expressions,
+   * projecting the fields of a given record type. */
+  public List<RexInputRef> identityProjects(final RelDataType rowType) {
+    return Lists.transform(rowType.getFieldList(), TO_INPUT_REF);
+  }
+
   //~ Methods ----------------------------------------------------------------
 
   /**
@@ -146,7 +160,8 @@ public class RexBuilder {
   public RexNode makeFieldAccess(RexNode expr, String fieldName,
       boolean caseSensitive) {
     final RelDataType type = expr.getType();
-    final RelDataTypeField field = type.getField(fieldName, caseSensitive);
+    final RelDataTypeField field =
+        type.getField(fieldName, caseSensitive, false);
     if (field == null) {
       throw Util.newInternal(
           "Type '" + type + "' has no field '"

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
index 63f8584..39c9ded 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
@@ -1151,6 +1151,19 @@ public abstract class SqlTypeUtil {
         ImmutableList.<String>of());
   }
 
+  /** Returns whether a type is flat. It is not flat if it is a record type that
+   * has one or more fields that are themselves record types. */
+  public static boolean isFlat(RelDataType type) {
+    if (type.isStruct()) {
+      for (RelDataTypeField field : type.getFieldList()) {
+        if (field.getType().isStruct()) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
   /**
    * Returns whether two types are comparable. They need to be scalar types of
    * the same family, or struct types whose fields are pairwise comparable.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
index b872b1d..768852e 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
@@ -104,15 +104,15 @@ public class SqlValidatorUtil {
   /**
    * Looks up a field with a given name, returning null if not found.
    *
+   * @param caseSensitive Whether match is case-sensitive
+   * @param elideRecord Whether to find fields nested within records
    * @param rowType    Row type
    * @param columnName Field name
    * @return Field, or null if not found
    */
-  public static RelDataTypeField lookupField(
-      boolean caseSensitive,
-      final RelDataType rowType,
-      String columnName) {
-    return rowType.getField(columnName, caseSensitive);
+  public static RelDataTypeField lookupField(boolean caseSensitive,
+      boolean elideRecord, final RelDataType rowType, String columnName) {
+    return rowType.getField(columnName, caseSensitive, elideRecord);
   }
 
   public static void checkCharsetAndCollateConsistentIfCharType(
@@ -343,14 +343,14 @@ public class SqlValidatorUtil {
 
   public static RelDataType createTypeFromProjection(RelDataType type,
       List<String> columnNameList, RelDataTypeFactory typeFactory,
-      boolean caseSensitive) {
+      boolean caseSensitive, boolean elideRecord) {
     // If the names in columnNameList and type have case-sensitive differences,
     // the resulting type will use those from type. These are presumably more
     // canonical.
     final List<RelDataTypeField> fields =
         new ArrayList<RelDataTypeField>(columnNameList.size());
     for (String name : columnNameList) {
-      RelDataTypeField field = type.getField(name, caseSensitive);
+      RelDataTypeField field = type.getField(name, caseSensitive, elideRecord);
       fields.add(type.getFieldList().get(field.getIndex()));
     }
     return typeFactory.createStructType(fields);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java b/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
index b34ea40..64f562f 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
@@ -62,6 +62,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.ReflectUtil;
 import org.apache.calcite.util.ReflectiveVisitDispatcher;
 import org.apache.calcite.util.ReflectiveVisitor;
@@ -69,11 +70,11 @@ import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.base.Function;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.SortedSetMultimap;
 
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -195,7 +196,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
   }
 
   private List<RexNode> restructureFields(RelDataType structuredType) {
-    List<RexNode> structuringExps = new ArrayList<RexNode>();
+    final List<RexNode> structuringExps = Lists.newArrayList();
     for (RelDataTypeField field : structuredType.getFieldList()) {
       // TODO:  row
       if (field.getType().getSqlTypeName() == SqlTypeName.STRUCTURED) {
@@ -339,11 +340,6 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
     return offset;
   }
 
-  protected RexNode flattenFieldAccesses(RexNode exp) {
-    RewriteRexShuttle shuttle = new RewriteRexShuttle();
-    return exp.accept(shuttle);
-  }
-
   public void rewriteRel(LogicalTableModify rel) {
     LogicalTableModify newRel =
         new LogicalTableModify(
@@ -402,7 +398,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
     RelNode newRel =
         RelOptUtil.createFilter(
             getNewForOldRel(rel.getInput()),
-            flattenFieldAccesses(rel.getCondition()));
+            rel.getCondition().accept(new RewriteRexShuttle()));
     setNewForOldRel(rel, newRel);
   }
 
@@ -412,7 +408,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
             rel.getCluster(),
             getNewForOldRel(rel.getLeft()),
             getNewForOldRel(rel.getRight()),
-            flattenFieldAccesses(rel.getCondition()),
+            rel.getCondition().accept(new RewriteRexShuttle()),
             rel.getJoinType(),
             rel.getVariablesStopped());
     setNewForOldRel(rel, newRel);
@@ -477,20 +473,17 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
   }
 
   public void rewriteRel(LogicalProject rel) {
-    final List<RexNode> flattenedExpList = new ArrayList<RexNode>();
-    final List<String> flattenedFieldNameList = new ArrayList<String>();
-    List<String> fieldNames = rel.getRowType().getFieldNames();
-    flattenProjections(
+    final List<Pair<RexNode, String>> flattenedExpList = Lists.newArrayList();
+    flattenProjections(new RewriteRexShuttle(),
         rel.getProjects(),
-        fieldNames,
+        rel.getRowType().getFieldNames(),
         "",
-        flattenedExpList,
-        flattenedFieldNameList);
+        flattenedExpList);
     RelNode newRel =
         RelOptUtil.createProject(
             getNewForOldRel(rel.getInput()),
             flattenedExpList,
-            flattenedFieldNameList);
+            false);
     setNewForOldRel(rel, newRel);
   }
 
@@ -506,29 +499,23 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
 
     // Convert the common expressions.
     final RexProgram program = rel.getProgram();
+    final RewriteRexShuttle shuttle = new RewriteRexShuttle();
     for (RexNode expr : program.getExprList()) {
-      programBuilder.registerInput(
-          flattenFieldAccesses(expr));
+      programBuilder.registerInput(expr.accept(shuttle));
     }
 
     // Convert the projections.
-    final List<RexNode> flattenedExpList = new ArrayList<RexNode>();
-    final List<String> flattenedFieldNameList = new ArrayList<String>();
+    final List<Pair<RexNode, String>> flattenedExpList = Lists.newArrayList();
     List<String> fieldNames = rel.getRowType().getFieldNames();
-    flattenProjections(
+    flattenProjections(new RewriteRexShuttle(),
         program.getProjectList(),
         fieldNames,
         "",
-        flattenedExpList,
-        flattenedFieldNameList);
+        flattenedExpList);
 
     // Register each of the new projections.
-    int i = -1;
-    for (RexNode flattenedExp : flattenedExpList) {
-      ++i;
-      programBuilder.addProject(
-          flattenedExp,
-          flattenedFieldNameList.get(i));
+    for (Pair<RexNode, String> flattenedExp : flattenedExpList) {
+      programBuilder.addProject(flattenedExp.left, flattenedExp.right);
     }
 
     // Translate the condition.
@@ -549,7 +536,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
             rel.getTraitSet(),
             newChild,
             newProgram,
-            Collections.<RelCollation>emptyList());
+            ImmutableList.<RelCollation>of());
     setNewForOldRel(rel, newRel);
   }
 
@@ -568,12 +555,11 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
     setNewForOldRel(rel, newRel);
   }
 
-  private void flattenProjections(
+  private void flattenProjections(RewriteRexShuttle shuttle,
       List<? extends RexNode> exps,
       List<String> fieldNames,
       String prefix,
-      List<RexNode> flattenedExps,
-      List<String> flattenedFieldNames) {
+      List<Pair<RexNode, String>> flattenedExps) {
     for (int i = 0; i < exps.size(); ++i) {
       RexNode exp = exps.get(i);
       String fieldName =
@@ -583,19 +569,14 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
       if (!prefix.equals("")) {
         fieldName = prefix + "$" + fieldName;
       }
-      flattenProjection(
-          exp,
-          fieldName,
-          flattenedExps,
-          flattenedFieldNames);
+      flattenProjection(shuttle, exp, fieldName, flattenedExps);
     }
   }
 
-  private void flattenProjection(
+  private void flattenProjection(RewriteRexShuttle shuttle,
       RexNode exp,
       String fieldName,
-      List<RexNode> flattenedExps,
-      List<String> flattenedFieldNames) {
+      List<Pair<RexNode, String>> flattenedExps) {
     if (exp.getType().isStruct()) {
       if (exp instanceof RexInputRef) {
         RexInputRef inputRef = (RexInputRef) exp;
@@ -612,10 +593,9 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
         for (int j = 0; j < n; ++j) {
           RelDataTypeField field = fieldList.get(j);
           flattenedExps.add(
-              new RexInputRef(
-                  newOffset + j,
-                  field.getType()));
-          flattenedFieldNames.add(fieldName);
+              Pair.<RexNode, String>of(
+                  new RexInputRef(newOffset + j, field.getType()),
+                  fieldName));
         }
       } else if (isConstructor(exp) || exp.isA(SqlKind.CAST)) {
         // REVIEW jvs 27-Feb-2005:  for cast, see corresponding note
@@ -625,8 +605,8 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
           // For object constructors, prepend a FALSE null
           // indicator.
           flattenedExps.add(
-              rexBuilder.makeLiteral(false));
-          flattenedFieldNames.add(fieldName);
+              Pair.<RexNode, String>of(rexBuilder.makeLiteral(false),
+                  fieldName));
         } else if (exp.isA(SqlKind.CAST)) {
           if (RexLiteral.isNullLiteral(
               ((RexCall) exp).operands.get(0))) {
@@ -634,41 +614,37 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
             // the correct number of null fields.
             flattenNullLiteral(
                 exp.getType(),
-                flattenedExps,
-                flattenedFieldNames);
+                flattenedExps);
             return;
           }
         }
-        flattenProjections(
+        flattenProjections(new RewriteRexShuttle(),
             call.getOperands(),
-            Collections.<String>nCopies(
-                call.getOperands().size(), null),
+            Collections.<String>nCopies(call.getOperands().size(), null),
             fieldName,
-            flattenedExps,
-            flattenedFieldNames);
+            flattenedExps);
       } else if (exp instanceof RexCall) {
         // NOTE jvs 10-Feb-2005:  This is a lame hack to keep special
         // functions which return row types working.
 
         int j = 0;
         for (RelDataTypeField field : exp.getType().getFieldList()) {
-          flattenedExps.add(rexBuilder.makeFieldAccess(exp, field.getIndex()));
-          flattenedFieldNames.add(fieldName + "$" + (j++));
+          flattenedExps.add(
+              Pair.of(rexBuilder.makeFieldAccess(exp, field.getIndex()),
+                  fieldName + "$" + (j++)));
         }
       } else {
         throw Util.needToImplement(exp);
       }
     } else {
-      exp = flattenFieldAccesses(exp);
-      flattenedExps.add(exp);
-      flattenedFieldNames.add(fieldName);
+      flattenedExps.add(
+          Pair.of(exp.accept(shuttle), fieldName));
     }
   }
 
   private void flattenNullLiteral(
       RelDataType type,
-      List<RexNode> flattenedExps,
-      List<String> flattenedFieldNames) {
+      List<Pair<RexNode, String>> flattenedExps) {
     RelDataType flattenedType =
         SqlTypeUtil.flattenRecordType(
             rexBuilder.getTypeFactory(),
@@ -676,10 +652,8 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
             null);
     for (RelDataTypeField field : flattenedType.getFieldList()) {
       flattenedExps.add(
-          rexBuilder.makeCast(
-              field.getType(),
-              rexBuilder.constantNull()));
-      flattenedFieldNames.add(field.getName());
+          Pair.of(rexBuilder.makeCast(field.getType(),
+                  rexBuilder.constantNull()), field.getName()));
     }
   }
 
@@ -694,11 +668,33 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
   }
 
   public void rewriteRel(LogicalTableScan rel) {
-    RelNode newRel =
-        rel.getTable().toRel(toRelContext);
+    RelNode newRel = rel.getTable().toRel(toRelContext);
+    if (!SqlTypeUtil.isFlat(rel.getRowType())) {
+      final List<Pair<RexNode, String>> flattenedExpList = Lists.newArrayList();
+      flattenInputs(rel.getRowType().getFieldList(),
+          rexBuilder.makeRangeReference(newRel),
+          flattenedExpList);
+      newRel =
+          RelOptUtil.createProject(newRel, flattenedExpList, false);
+    }
     setNewForOldRel(rel, newRel);
   }
 
+  /** Generates expressions that reference the flattened input fields from
+   * a given row type. */
+  private void flattenInputs(List<RelDataTypeField> fieldList, RexNode prefix,
+      List<Pair<RexNode, String>> flattenedExpList) {
+    for (RelDataTypeField field : fieldList) {
+      final RexNode ref =
+          rexBuilder.makeFieldAccess(prefix, field.getIndex());
+      if (field.getType().isStruct()) {
+        flattenInputs(field.getType().getFieldList(), ref, flattenedExpList);
+      } else {
+        flattenedExpList.add(Pair.of(ref, field.getName()));
+      }
+    }
+  }
+
   //~ Inner Interfaces -------------------------------------------------------
 
   /** Mix-in interface for relational expressions that know how to
@@ -747,8 +743,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
 
   /** Shuttle that rewrites scalar expressions. */
   private class RewriteRexShuttle extends RexShuttle {
-    // override RexShuttle
-    public RexNode visitInputRef(RexInputRef input) {
+    @Override public RexNode visitInputRef(RexInputRef input) {
       final int oldIndex = input.getIndex();
       final int newIndex = getNewForOldInput(oldIndex);
 
@@ -768,8 +763,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
       return type.getFieldList().get(0).getType();
     }
 
-    // override RexShuttle
-    public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
+    @Override public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
       // walk down the field access path expression, calculating
       // the desired input number
       int iInput = 0;
@@ -805,8 +799,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
       }
     }
 
-    // override RexShuttle
-    public RexNode visitCall(RexCall rexCall) {
+    @Override public RexNode visitCall(RexCall rexCall) {
       if (rexCall.isA(SqlKind.CAST)) {
         RexNode input = rexCall.getOperands().get(0).accept(this);
         RelDataType targetType = removeDistinct(rexCall.getType());
@@ -837,13 +830,8 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
         RexBuilder rexBuilder,
         SqlOperator op,
         List<RexNode> exprs) {
-      List<RexNode> flattenedExps = new ArrayList<RexNode>();
-      flattenProjections(
-          exprs,
-          null,
-          "",
-          flattenedExps,
-          new ArrayList<String>());
+      final List<Pair<RexNode, String>> flattenedExps = Lists.newArrayList();
+      flattenProjections(this, exprs, null, "", flattenedExps);
       int n = flattenedExps.size() / 2;
       boolean negate = false;
       if (op.getKind() == SqlKind.NOT_EQUALS) {
@@ -859,8 +847,8 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
         RexNode comparison =
             rexBuilder.makeCall(
                 op,
-                flattenedExps.get(i),
-                flattenedExps.get(i + n));
+                flattenedExps.get(i).left,
+                flattenedExps.get(i + n).left);
         if (conjunction == null) {
           conjunction = comparison;
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index b96fb6d..b7b1d39 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -2008,9 +2008,9 @@ public class JdbcTest {
         .query("select * from \"hr\".\"emps\",\n"
             + " LATERAL (select * from \"hr\".\"depts\" where \"emps\".\"deptno\" = \"depts\".\"deptno\")")
         .returnsUnordered(
-            "empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000; deptno0=10; name0=Sales; employees=[Employee [empid: 100, deptno: 10, name: Bill], Employee [empid: 150, deptno: 10, name: Sebastian]]",
-            "empid=110; deptno=10; name=Theodore; salary=11500.0; commission=250; deptno0=10; name0=Sales; employees=[Employee [empid: 100, deptno: 10, name: Bill], Employee [empid: 150, deptno: 10, name: Sebastian]]",
-            "empid=150; deptno=10; name=Sebastian; salary=7000.0; commission=null; deptno0=10; name0=Sales; employees=[Employee [empid: 100, deptno: 10, name: Bill], Employee [empid: 150, deptno: 10, name: Sebastian]]");
+            "empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000; deptno0=10; name0=Sales; employees=[Employee [empid: 100, deptno: 10, name: Bill], Employee [empid: 150, deptno: 10, name: Sebastian]]; location=Location [x: -122, y: 38]",
+            "empid=110; deptno=10; name=Theodore; salary=11500.0; commission=250; deptno0=10; name0=Sales; employees=[Employee [empid: 100, deptno: 10, name: Bill], Employee [empid: 150, deptno: 10, name: Sebastian]]; location=Location [x: -122, y: 38]",
+            "empid=150; deptno=10; name=Sebastian; salary=7000.0; commission=null; deptno0=10; name0=Sales; employees=[Employee [empid: 100, deptno: 10, name: Bill], Employee [empid: 150, deptno: 10, name: Sebastian]]; location=Location [x: -122, y: 38]");
   }
 
   /** Per SQL std, UNNEST is implicitly LATERAL. */
@@ -2503,10 +2503,10 @@ public class JdbcTest {
         .query(
             "select * from \"hr\".\"emps\", \"hr\".\"depts\" where \"emps\".\"empid\" < 140 and \"depts\".\"deptno\" > 20")
         .returnsUnordered(
-            "empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000; deptno0=30; name0=Marketing; employees=[]",
-            "empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000; deptno0=40; name0=HR; employees=[Employee [empid: 200, deptno: 20, name: Eric]]",
-            "empid=110; deptno=10; name=Theodore; salary=11500.0; commission=250; deptno0=30; name0=Marketing; employees=[]",
-            "empid=110; deptno=10; name=Theodore; salary=11500.0; commission=250; deptno0=40; name0=HR; employees=[Employee [empid: 200, deptno: 20, name: Eric]]");
+            "empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000; deptno0=30; name0=Marketing; employees=[]; location=Location [x: 0, y: 52]",
+            "empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000; deptno0=40; name0=HR; employees=[Employee [empid: 200, deptno: 20, name: Eric]]; location=null",
+            "empid=110; deptno=10; name=Theodore; salary=11500.0; commission=250; deptno0=30; name0=Marketing; employees=[]; location=Location [x: 0, y: 52]",
+            "empid=110; deptno=10; name=Theodore; salary=11500.0; commission=250; deptno0=40; name0=HR; employees=[Employee [empid: 200, deptno: 20, name: Eric]]; location=null");
   }
 
   @Test public void testDistinctCountSimple() {
@@ -2592,9 +2592,9 @@ public class JdbcTest {
             + "  select \"deptno\" from \"hr\".\"emps\"\n"
             + "  where \"empid\" < 150)")
         .convertContains(""
-            + "LogicalProject(deptno=[$0], name=[$1], employees=[$2])\n"
-            + "  LogicalJoin(condition=[=($3, $4)], joinType=[inner])\n"
-            + "    LogicalProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$0])\n"
+            + "LogicalProject(deptno=[$0], name=[$1], employees=[$2], location=[$3])\n"
+            + "  LogicalJoin(condition=[=($4, $5)], joinType=[inner])\n"
+            + "    LogicalProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$0])\n"
             + "      EnumerableTableScan(table=[[hr, depts]])\n"
             + "    LogicalAggregate(group=[{0}])\n"
             + "      LogicalProject(deptno=[$1])\n"
@@ -2602,14 +2602,14 @@ public class JdbcTest {
             + "          LogicalProject(empid=[$0], deptno=[$1])\n"
             + "            EnumerableTableScan(table=[[hr, emps]])")
         .explainContains(""
-            + "EnumerableCalc(expr#0..3=[{inputs}], proj#0..2=[{exprs}])\n"
-            + "  EnumerableSemiJoin(condition=[=($3, $4)], joinType=[inner])\n"
-            + "    EnumerableCalc(expr#0..2=[{inputs}], proj#0..2=[{exprs}], $f3=[$t0])\n"
+            + "EnumerableCalc(expr#0..4=[{inputs}], proj#0..3=[{exprs}])\n"
+            + "  EnumerableSemiJoin(condition=[=($4, $5)], joinType=[inner])\n"
+            + "    EnumerableCalc(expr#0..3=[{inputs}], proj#0..3=[{exprs}], $f4=[$t0])\n"
             + "      EnumerableTableScan(table=[[hr, depts]])\n"
             + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[150], expr#6=[<($t0, $t5)], deptno=[$t1], $condition=[$t6])\n"
             + "      EnumerableTableScan(table=[[hr, emps]])")
         .returnsUnordered(
-            "deptno=10; name=Sales; employees=[Employee [empid: 100, deptno: 10, name: Bill], Employee [empid: 150, deptno: 10, name: Sebastian]]");
+            "deptno=10; name=Sales; employees=[Employee [empid: 100, deptno: 10, name: Bill], Employee [empid: 150, deptno: 10, name: Sebastian]]; location=Location [x: -122, y: 38]");
   }
 
   /** A difficult query: an IN list so large that the planner promotes it
@@ -6054,9 +6054,11 @@ public class JdbcTest {
       new Employee(110, 10, "Theodore", 11500, 250),
     };
     public final Department[] depts = {
-      new Department(10, "Sales", Arrays.asList(emps[0], emps[2])),
-      new Department(30, "Marketing", Collections.<Employee>emptyList()),
-      new Department(40, "HR", Collections.singletonList(emps[1])),
+      new Department(10, "Sales", Arrays.asList(emps[0], emps[2]),
+          new Location(-122, 38)),
+      new Department(30, "Marketing", Collections.<Employee>emptyList(),
+          new Location(0, 52)),
+      new Department(40, "HR", Collections.singletonList(emps[1]), null),
     };
     public final Dependent[] dependents = {
       new Dependent(10, "Michael"),
@@ -6112,32 +6114,34 @@ public class JdbcTest {
     public final int deptno;
     public final String name;
     public final List<Employee> employees;
+    public final Location location;
 
-    public Department(
-        int deptno, String name, List<Employee> employees) {
+    public Department(int deptno, String name, List<Employee> employees,
+        Location location) {
       this.deptno = deptno;
       this.name = name;
       this.employees = employees;
+      this.location = location;
     }
 
 
     public String toString() {
       return "Department [deptno: " + deptno + ", name: " + name
-          + ", employees: " + employees + "]";
+          + ", employees: " + employees + ", location: " + location + "]";
     }
   }
 
   public static class Location {
-    public final int locid;
-    public final String name;
+    public final int x;
+    public final int y;
 
-    public Location(int locid, String name) {
-      this.locid = locid;
-      this.name = name;
+    public Location(int x, int y) {
+      this.x = x;
+      this.y = y;
     }
 
     @Override public String toString() {
-      return "Location [locid: " + locid + ", name: " + name + "]";
+      return "Location [x: " + x + ", y: " + y + "]";
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
index adf4565..ed8e9a6 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -76,6 +76,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
   protected final RelDataTypeFactory typeFactory;
   private final boolean caseSensitive;
+  private final boolean elideRecord = true;
   private final Map<List<String>, MockTable> tables;
   protected final Map<String, MockSchema> schemas;
   private RelDataType addressType;
@@ -328,7 +329,8 @@ public class MockCatalogReader implements Prepare.CatalogReader {
   }
 
   public RelDataTypeField field(RelDataType rowType, String alias) {
-    return SqlValidatorUtil.lookupField(caseSensitive, rowType, alias);
+    return SqlValidatorUtil.lookupField(caseSensitive, elideRecord, rowType,
+        alias);
   }
 
   public int fieldOrdinal(RelDataType rowType, String alias) {
@@ -347,7 +349,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
   public RelDataType createTypeFromProjection(final RelDataType type,
       final List<String> columnNameList) {
     return SqlValidatorUtil.createTypeFromProjection(type, columnNameList,
-        typeFactory, caseSensitive);
+        typeFactory, caseSensitive, elideRecord);
   }
 
   private static List<RelCollation> deduceMonotonicity(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index a1becf6..8a4cbc3 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -6488,6 +6488,20 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "RecordType(INTEGER NOT NULL X, VARCHAR(20) NOT NULL EMAIL, INTEGER NOT NULL Y) NOT NULL");
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-xxx">CALCITE-xxx,
+   * "Unexpected upper-casing of keywords when using java lexer"</a>. */
+  @Test public void testRecordTypeElided() {
+    checkResultType(
+        "SELECT contact.x, contact.coord.y FROM customer.contact",
+        "RecordType(INTEGER NOT NULL X, INTEGER NOT NULL Y) NOT NULL");
+
+    // Qualifying with schema is OK.
+    checkResultType(
+        "SELECT customer.contact.x, customer.contact.email, contact.coord.y FROM customer.contact",
+        "RecordType(INTEGER NOT NULL X, VARCHAR(20) NOT NULL EMAIL, INTEGER NOT NULL Y) NOT NULL");
+  }
+
   @Test public void testSample() {
     // applied to table
     check("SELECT * FROM emp TABLESAMPLE SUBSTITUTE('foo')");

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
index a725e52..962c83c 100644
--- a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
+++ b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
@@ -41,7 +41,7 @@ public class EnumerableCorrelateTest {
             "EnumerableCalc(expr#0..4=[{inputs}], empid=[$t0], name=[$t2])\n"
             + "  EnumerableSemiJoin(condition=[=($1, $5)], joinType=[inner])\n"
             + "    EnumerableTableScan(table=[[s, emps]])\n"
-            + "    EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], $f01=[$t0], $f0=[$t4])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], $f01=[$t0], $f0=[$t5])\n"
             + "      EnumerableJoin(condition=[=($0, $1)], joinType=[inner])\n"
             + "        EnumerableAggregate(group=[{0}])\n"
             + "          EnumerableCalc(expr#0..4=[{inputs}], $f0=[$t1])\n"
@@ -63,7 +63,7 @@ public class EnumerableCorrelateTest {
             + "  EnumerableCorrelate(correlation=[$cor0], joinType=[LEFT], requiredColumns=[{1}])\n"
             + "    EnumerableTableScan(table=[[s, emps]])\n"
             + "    EnumerableAggregate(group=[{}], agg#0=[MIN($0)])\n"
-            + "      EnumerableCalc(expr#0..2=[{inputs}], expr#3=[true], expr#4=[$cor0], expr#5=[$t4.deptno], expr#6=[=($t0, $t5)], $f0=[$t3], $condition=[$t6])\n"
+            + "      EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], expr#5=[$cor0], expr#6=[$t5.deptno], expr#7=[=($t0, $t6)], $f0=[$t4], $condition=[$t7])\n"
             + "        EnumerableTableScan(table=[[s, depts]])")
         .returnsUnordered(
             "empid=100; name=Bill",

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
index 9d79e86..82137f5 100644
--- a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
@@ -555,7 +555,7 @@ public class PlannerTest {
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
     assertThat(toString(transform), containsString(
-        "EnumerableJoin(condition=[=($0, $3)], joinType=[inner])"));
+        "EnumerableJoin(condition=[=($0, $5)], joinType=[inner])"));
   }
 
   /** Test case for
@@ -574,14 +574,15 @@ public class PlannerTest {
         "select * from \"emps\" as e\n"
             + "left join \"depts\" as d using (\"deptno\")\n"
             + "join \"dependents\" as p on e.\"empid\" = p.\"empid\"",
-        "EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], empid0=[$8], name1=[$9])\n"
-            + "  EnumerableProject(empid=[$2], deptno=[$3], name=[$4], salary=[$5], commission=[$6], deptno0=[$7], name0=[$8], employees=[$9], empid0=[$0], name1=[$1])\n"
+        "EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], location=[$8], location9=[$9], empid0=[$10], name1=[$11])\n"
+            + "  EnumerableProject(empid=[$2], deptno=[$3], name=[$4], salary=[$5], commission=[$6], deptno0=[$7], name0=[$8], employees=[$9], x=[$10], y=[$11], empid0=[$0], name1=[$1])\n"
             + "    EnumerableJoin(condition=[=($0, $2)], joinType=[inner])\n"
             + "      EnumerableTableScan(table=[[hr, dependents]])\n"
-            + "      EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7])\n"
+            + "      EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], x=[$8], y=[$9])\n"
             + "        EnumerableJoin(condition=[=($1, $5)], joinType=[left])\n"
             + "          EnumerableTableScan(table=[[hr, emps]])\n"
-            + "          EnumerableTableScan(table=[[hr, depts]])");
+            + "          EnumerableProject(deptno=[$0], name=[$1], employees=[$2], x=[$3.x], y=[$3.y])\n"
+            + "            EnumerableTableScan(table=[[hr, depts]])");
   }
 
   /** It would probably be OK to transform
@@ -595,14 +596,15 @@ public class PlannerTest {
         "select * from \"emps\" as e\n"
             + "right join \"depts\" as d using (\"deptno\")\n"
             + "join \"dependents\" as p on e.\"empid\" = p.\"empid\"",
-        "EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], empid0=[$8], name1=[$9])\n"
-            + "  EnumerableProject(empid=[$2], deptno=[$3], name=[$4], salary=[$5], commission=[$6], deptno0=[$7], name0=[$8], employees=[$9], empid0=[$0], name1=[$1])\n"
+        "EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], location=[$8], location9=[$9], empid0=[$10], name1=[$11])\n"
+            + "  EnumerableProject(empid=[$2], deptno=[$3], name=[$4], salary=[$5], commission=[$6], deptno0=[$7], name0=[$8], employees=[$9], x=[$10], y=[$11], empid0=[$0], name1=[$1])\n"
             + "    EnumerableJoin(condition=[=($0, $2)], joinType=[inner])\n"
             + "      EnumerableTableScan(table=[[hr, dependents]])\n"
-            + "      EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7])\n"
-            + "        EnumerableJoin(condition=[=($1, $5)], joinType=[right])\n"
-            + "          EnumerableTableScan(table=[[hr, emps]])\n"
-            + "          EnumerableTableScan(table=[[hr, depts]])");
+            + "      EnumerableProject(empid=[$5], deptno=[$6], name=[$7], salary=[$8], commission=[$9], deptno0=[$0], name0=[$1], employees=[$2], x=[$3], y=[$4])\n"
+            + "        EnumerableJoin(condition=[=($0, $6)], joinType=[left])\n"
+            + "          EnumerableProject(deptno=[$0], name=[$1], employees=[$2], x=[$3.x], y=[$3.y])\n"
+            + "            EnumerableTableScan(table=[[hr, depts]])\n"
+            + "          EnumerableTableScan(table=[[hr, emps]])");
   }
 
   /** Tests that a relation (dependents) that is on the null-generating side of
@@ -612,14 +614,15 @@ public class PlannerTest {
         "select * from \"emps\" as e\n"
             + "join \"depts\" as d using (\"deptno\")\n"
             + "right join \"dependents\" as p on e.\"empid\" = p.\"empid\"",
-        "EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], empid0=[$8], name1=[$9])\n"
-            + "  EnumerableProject(empid=[$2], deptno=[$3], name=[$4], salary=[$5], commission=[$6], deptno0=[$7], name0=[$8], employees=[$9], empid0=[$0], name1=[$1])\n"
+        "EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], location=[$8], location9=[$9], empid0=[$10], name1=[$11])\n"
+            + "  EnumerableProject(empid=[$2], deptno=[$3], name=[$4], salary=[$5], commission=[$6], deptno0=[$7], name0=[$8], employees=[$9], x=[$10], y=[$11], empid0=[$0], name1=[$1])\n"
             + "    EnumerableJoin(condition=[=($0, $2)], joinType=[left])\n"
             + "      EnumerableTableScan(table=[[hr, dependents]])\n"
-            + "      EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7])\n"
+            + "      EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], x=[$8], y=[$9])\n"
             + "        EnumerableJoin(condition=[=($1, $5)], joinType=[inner])\n"
             + "          EnumerableTableScan(table=[[hr, emps]])\n"
-            + "          EnumerableTableScan(table=[[hr, depts]])");
+            + "          EnumerableProject(deptno=[$0], name=[$1], employees=[$2], x=[$3.x], y=[$3.y])\n"
+            + "            EnumerableTableScan(table=[[hr, depts]])");
   }
 
   private void checkHeuristic(String sql, String expected) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/core/src/test/resources/sql/misc.oq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.oq b/core/src/test/resources/sql/misc.oq
index 21eace7..bb32ae0 100644
--- a/core/src/test/resources/sql/misc.oq
+++ b/core/src/test/resources/sql/misc.oq
@@ -200,7 +200,7 @@ EnumerableCalc(expr#0..4=[{inputs}], expr#5=[CAST($t2):VARCHAR(1) CHARACTER SET
   EnumerableJoin(condition=[=($1, $3)], joinType=[inner])
     EnumerableCalc(expr#0..4=[{inputs}], proj#0..2=[{exprs}])
       EnumerableTableScan(table=[[hr, emps]])
-    EnumerableCalc(expr#0..2=[{inputs}], proj#0..1=[{exprs}])
+    EnumerableCalc(expr#0..3=[{inputs}], proj#0..1=[{exprs}])
       EnumerableTableScan(table=[[hr, depts]])
 !plan
 
@@ -224,7 +224,7 @@ EnumerableCalc(expr#0..4=[{inputs}], expr#5=[CAST($t2):VARCHAR(1) CHARACTER SET
   EnumerableJoin(condition=[=($1, $3)], joinType=[inner])
     EnumerableCalc(expr#0..4=[{inputs}], proj#0..2=[{exprs}])
       EnumerableTableScan(table=[[hr, emps]])
-    EnumerableCalc(expr#0..2=[{inputs}], proj#0..1=[{exprs}])
+    EnumerableCalc(expr#0..3=[{inputs}], proj#0..1=[{exprs}])
       EnumerableTableScan(table=[[hr, depts]])
 !plan
 
@@ -244,7 +244,7 @@ where exists (
 !ok
 EnumerableSemiJoin(condition=[=($1, $5)], joinType=[inner])
   EnumerableTableScan(table=[[hr, emps]])
-  EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], $f01=[$t0], $f0=[$t4])
+  EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], $f01=[$t0], $f0=[$t5])
     EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
       EnumerableAggregate(group=[{0}])
         EnumerableCalc(expr#0..4=[{inputs}], $f0=[$t1])
@@ -269,7 +269,7 @@ EnumerableCalc(expr#0..6=[{inputs}], expr#7=[IS NOT NULL($t6)], expr#8=[NOT($t7)
   EnumerableJoin(condition=[=($1, $5)], joinType=[left])
     EnumerableTableScan(table=[[hr, emps]])
     EnumerableAggregate(group=[{0}], agg#0=[MIN($1)])
-      EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], $f01=[$t0], $f0=[$t4])
+      EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], $f01=[$t0], $f0=[$t5])
         EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
           EnumerableAggregate(group=[{0}])
             EnumerableCalc(expr#0..4=[{inputs}], $f0=[$t1])
@@ -301,20 +301,20 @@ EnumerableCalc(expr#0..7=[{inputs}], expr#8=[IS NOT NULL($t5)], expr#9=[NOT($t8)
       EnumerableJoin(condition=[=($1, $5)], joinType=[left])
         EnumerableTableScan(table=[[hr, emps]])
         EnumerableAggregate(group=[{0}], agg#0=[MIN($1)])
-          EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], $f01=[$t0], $f0=[$t4])
+          EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], $f01=[$t0], $f0=[$t5])
             EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
               EnumerableAggregate(group=[{0}])
                 EnumerableCalc(expr#0..4=[{inputs}], $f0=[$t1])
                   EnumerableTableScan(table=[[hr, emps]])
               EnumerableTableScan(table=[[hr, depts]])
     EnumerableAggregate(group=[{0}], agg#0=[MIN($1)])
-      EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], expr#5=[90], expr#6=[+($t1, $t5)], expr#7=[CAST($t0):INTEGER NOT NULL], expr#8=[=($t6, $t7)], $f01=[$t0], $f0=[$t4], $condition=[$t8])
+      EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], expr#6=[90], expr#7=[+($t1, $t6)], expr#8=[CAST($t0):INTEGER NOT NULL], expr#9=[=($t7, $t8)], $f01=[$t0], $f0=[$t5], $condition=[$t9])
         EnumerableJoin(condition=[true], joinType=[inner])
           EnumerableAggregate(group=[{0}])
             EnumerableCalc(expr#0..4=[{inputs}], $f0=[$t0])
               EnumerableSemiJoin(condition=[=($1, $5)], joinType=[inner])
                 EnumerableTableScan(table=[[hr, emps]])
-                EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], $f01=[$t0], $f0=[$t4])
+                EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], $f01=[$t0], $f0=[$t5])
                   EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
                     EnumerableAggregate(group=[{0}])
                       EnumerableCalc(expr#0..4=[{inputs}], $f0=[$t1])
@@ -386,7 +386,7 @@ EnumerableAggregate(group=[{}], C=[COUNT()])
     EnumerableJoin(condition=[true], joinType=[inner])
       EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], DUMMY=[$t5])
         EnumerableTableScan(table=[[hr, emps]])
-      EnumerableCalc(expr#0..2=[{inputs}], expr#3=[0], DUMMY=[$t3])
+      EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], DUMMY=[$t4])
         EnumerableTableScan(table=[[hr, depts]])
 !plan
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/175d0705/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkTableScan.java
----------------------------------------------------------------------
diff --git a/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkTableScan.java b/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkTableScan.java
index 8c990f1..f8dabe6 100644
--- a/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkTableScan.java
+++ b/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkTableScan.java
@@ -104,7 +104,7 @@ public class SplunkTableScan
         getCluster().getTypeFactory().builder();
     for (String field : fieldList) {
       // REVIEW: is case-sensitive match what we want here?
-      builder.add(table.getRowType().getField(field, true));
+      builder.add(table.getRowType().getField(field, true, false));
     }
     return builder.build();
   }