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 2016/09/16 04:48:18 UTC

[4/6] calcite git commit: Complete [CALCITE-1208] by giving "resolve" a call-back, so it can make multiple matches

http://git-wip-us.apache.org/repos/asf/calcite/blob/0938c7b6/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 0efdaf2..d380c1b 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -2203,20 +2203,15 @@ public class SqlToRelConverter {
       String originalRelName = lookup.getOriginalRelName();
       String originalFieldName = fieldAccess.getField().getName();
 
-      int[] nsIndexes = {-1};
-      final SqlValidatorScope[] ancestorScopes = {null};
-      SqlValidatorNamespace foundNs =
-          lookup.bb.scope.resolve(
-              ImmutableList.of(originalRelName),
-              ancestorScopes,
-              nsIndexes);
-
-      assert foundNs != null;
-      assert nsIndexes.length == 1;
-
-      int childNamespaceIndex = nsIndexes[0];
-
-      SqlValidatorScope ancestorScope = ancestorScopes[0];
+      SqlValidatorScope.ResolvedImpl resolved =
+          new SqlValidatorScope.ResolvedImpl();
+      lookup.bb.scope.resolve(ImmutableList.of(originalRelName), false,
+          resolved);
+      assert resolved.count() == 1;
+      final SqlValidatorScope.Resolve resolve = resolved.only();
+      final SqlValidatorNamespace foundNs = resolve.namespace;
+      final int childNamespaceIndex = resolve.path.steps().get(0).i;
+      final SqlValidatorScope ancestorScope = resolve.scope;
       boolean correlInCurrentScope = ancestorScope == bb.scope;
 
       if (!correlInCurrentScope) {
@@ -2309,18 +2304,12 @@ public class SqlToRelConverter {
       DeferredLookup lookup = mapCorrelToDeferred.get(correlName);
       String originalRelName = lookup.getOriginalRelName();
 
-      int[] nsIndexes = {-1};
-      final SqlValidatorScope[] ancestorScopes = {null};
-      SqlValidatorNamespace foundNs =
-          lookup.bb.scope.resolve(
-              ImmutableList.of(originalRelName),
-              ancestorScopes,
-              nsIndexes);
-
-      assert foundNs != null;
-      assert nsIndexes.length == 1;
+      final SqlValidatorScope.ResolvedImpl resolved =
+          new SqlValidatorScope.ResolvedImpl();
+      lookup.bb.scope.resolve(ImmutableList.of(originalRelName), false,
+          resolved);
 
-      SqlValidatorScope ancestorScope = ancestorScopes[0];
+      SqlValidatorScope ancestorScope = resolved.only().scope;
 
       // If the correlated reference is in a scope that's "above" the
       // subquery, then this is a correlated subquery.
@@ -3900,24 +3889,24 @@ public class SqlToRelConverter {
         }
         return Pair.of(node, null);
       }
-      int[] offsets = {-1};
-      final SqlValidatorScope[] ancestorScopes = {null};
-      SqlValidatorNamespace foundNs =
-          scope.resolve(qualified.prefix(), ancestorScopes, offsets);
-      if (foundNs == null) {
+      final SqlValidatorScope.ResolvedImpl resolved =
+          new SqlValidatorScope.ResolvedImpl();
+      scope.resolve(qualified.prefix(), false, resolved);
+      if (!(resolved.count() == 1)) {
         return null;
       }
+      final SqlValidatorScope.Resolve resolve = resolved.only();
+      final SqlValidatorNamespace foundNs = resolve.namespace;
 
       // Found in current query's from list.  Find which from item.
       // We assume that the order of the from clause items has been
       // preserved.
-      SqlValidatorScope ancestorScope = ancestorScopes[0];
+      final SqlValidatorScope ancestorScope = resolve.scope;
       boolean isParent = ancestorScope != scope;
       if ((inputs != null) && !isParent) {
-        int offset = offsets[0];
         final LookupContext rels =
             new LookupContext(this, inputs, systemFieldList.size());
-        final RexNode node = lookup(offset, rels);
+        final RexNode node = lookup(resolve.path.steps().get(0).i, rels);
         if (node == null) {
           return null;
         } else {
@@ -3932,20 +3921,20 @@ public class SqlToRelConverter {
             new DeferredLookup(this, qualified.identifier.names.get(0));
         final CorrelationId correlId = cluster.createCorrel();
         mapCorrelToDeferred.put(correlId, lookup);
-        if (offsets[0] < 0) {
+        if (resolve.path.steps().get(0).i < 0) {
           return Pair.of(rexBuilder.makeCorrel(foundNs.getRowType(), correlId),
               null);
         } else {
           final RelDataTypeFactory.FieldInfoBuilder builder =
               typeFactory.builder();
-          final ListScope ancestorScope1 = (ListScope) ancestorScopes[0];
+          final ListScope ancestorScope1 = (ListScope) resolve.scope;
           final ImmutableMap.Builder<String, Integer> fields =
               ImmutableMap.builder();
           int i = 0;
           int offset = 0;
           for (SqlValidatorNamespace c : ancestorScope1.getChildren()) {
             builder.addAll(c.getRowType().getFieldList());
-            if (i == offsets[0]) {
+            if (i == resolve.path.steps().get(0).i) {
               for (RelDataTypeField field : c.getRowType().getFieldList()) {
                 fields.put(field.getName(), field.getIndex() + offset);
               }

http://git-wip-us.apache.org/repos/asf/calcite/blob/0938c7b6/core/src/main/java/org/apache/calcite/util/ImmutableIntList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ImmutableIntList.java b/core/src/main/java/org/apache/calcite/util/ImmutableIntList.java
index 78b3dbe..90cbdb0 100644
--- a/core/src/main/java/org/apache/calcite/util/ImmutableIntList.java
+++ b/core/src/main/java/org/apache/calcite/util/ImmutableIntList.java
@@ -223,8 +223,12 @@ public class ImmutableIntList extends FlatLists.AbstractFlatList<Integer> {
     return -1;
   }
 
+  @Override public ImmutableIntList append(Integer e) {
+    return append((int) e);
+  }
+
   /** Returns a copy of this list with one element added. */
-  public ImmutableIntList add(int element) {
+  public ImmutableIntList append(int element) {
     if (ints.length == 0) {
       return of(element);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/0938c7b6/core/src/main/java/org/apache/calcite/util/Pair.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Pair.java b/core/src/main/java/org/apache/calcite/util/Pair.java
index a3d509a..bb2521e 100644
--- a/core/src/main/java/org/apache/calcite/util/Pair.java
+++ b/core/src/main/java/org/apache/calcite/util/Pair.java
@@ -73,6 +73,11 @@ public class Pair<T1, T2>
     return new Pair<>(left, right);
   }
 
+  /** Creates a {@code Pair} from a {@link java.util.Map.Entry}. */
+  public static <K, V> Pair<K, V> of(Map.Entry<K, V> entry) {
+    return of(entry.getKey(), entry.getValue());
+  }
+
   //~ Methods ----------------------------------------------------------------
 
   public boolean equals(Object obj) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/0938c7b6/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index bab7a4a..729fa13 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -198,6 +198,7 @@ ModifiableViewMustBeBasedOnSingleTable=Modifiable view must be based on a single
 MoreThanOneMappedColumn=View is not modifiable. More than one expression maps to column ''{0}'' of base table ''{1}''
 NoValueSuppliedForViewColumn=View is not modifiable. No value is supplied for NOT NULL column ''{0}'' of base table ''{1}''
 TableNotFound=Table ''{0}'' not found
+StarRequiresRecordType=Not a record type. The ''*'' operator requires a record
 FilterMustBeBoolean=FILTER expression must be of type BOOLEAN
 CannotStreamResultsForNonStreamingInputs=Cannot stream results of a query with no streaming inputs: ''{0}''. At least one input should be convertible to a stream
 SelectMissingFrom=SELECT must have a FROM clause

http://git-wip-us.apache.org/repos/asf/calcite/blob/0938c7b6/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 7a3ea1a..a65f797 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -2126,6 +2126,11 @@ public class SqlParserTest {
                 + "FROM `EMP`");
   }
 
+  @Test public void testTableStarColumnFails() {
+    sql("select emp.*^.^xx from emp")
+        .fails("(?s).*Encountered \".\" .*");
+  }
+
   @Test public void testNotExists() {
     check(
         "select * from dept where not not exists (select * from emp) and true",
@@ -2639,6 +2644,16 @@ public class SqlParserTest {
             + "FROM `EMP`");
   }
 
+  @Test public void testCompoundStar() {
+    final String sql = "select sales.emp.address.zipcode,\n"
+        + " sales.emp.address.*\n"
+        + "from sales.emp";
+    final String expected = "SELECT `SALES`.`EMP`.`ADDRESS`.`ZIPCODE`,"
+        + " `SALES`.`EMP`.`ADDRESS`.*\n"
+        + "FROM `SALES`.`EMP`";
+    sql(sql).ok(expected);
+  }
+
   @Test public void testSelectDistinct() {
     check(
         "select distinct foo from bar",

http://git-wip-us.apache.org/repos/asf/calcite/blob/0938c7b6/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
index 2a3c02f..03200f0 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
@@ -81,7 +81,9 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
   private static final List<String> SCHEMAS =
       Arrays.asList(
           "CATALOG(CATALOG)",
+          "SCHEMA(CATALOG.DYNAMIC)",
           "SCHEMA(CATALOG.SALES)",
+          "SCHEMA(CATALOG.STRUCT)",
           "SCHEMA(CATALOG.CUSTOMER)");
 
   private static final List<String> AB_TABLES =

http://git-wip-us.apache.org/repos/asf/calcite/blob/0938c7b6/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 d61b440..23719d4 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -34,15 +34,14 @@ import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.rel.type.DynamicRecordTypeImpl;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataType.StructKind;
 import org.apache.calcite.rel.type.RelDataTypeComparability;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
 import org.apache.calcite.rel.type.RelDataTypeFamily;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
 import org.apache.calcite.rel.type.RelDataTypePrecedenceList;
 import org.apache.calcite.rel.type.RelRecordType;
+import org.apache.calcite.rel.type.StructKind;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
@@ -115,7 +114,6 @@ 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;
@@ -165,7 +163,16 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     final RelDataType booleanType =
         typeFactory.createSqlType(SqlTypeName.BOOLEAN);
     final RelDataType rectilinearCoordType =
-        typeFactory.builder().add("X", intType).add("Y", intType).build();
+        typeFactory.builder()
+            .add("X", intType)
+            .add("Y", intType)
+            .build();
+    final RelDataType rectilinearPeekCoordType =
+        typeFactory.builder()
+            .add("X", intType)
+            .add("Y", intType)
+            .kind(StructKind.PEEK_FIELDS)
+            .build();
     final RelDataType empRecordType =
         typeFactory.builder()
             .add("EMPNO", intType)
@@ -183,8 +190,8 @@ public class MockCatalogReader implements Prepare.CatalogReader {
             Arrays.asList(
                 new RelDataTypeFieldImpl("STREET", 0, varchar20Type),
                 new RelDataTypeFieldImpl("CITY", 1, varchar20Type),
-                new RelDataTypeFieldImpl("ZIP", 1, intType),
-                new RelDataTypeFieldImpl("STATE", 1, varchar20Type)),
+                new RelDataTypeFieldImpl("ZIP", 2, intType),
+                new RelDataTypeFieldImpl("STATE", 3, varchar20Type)),
             RelDataTypeComparability.NONE);
 
     // Register "SALES" schema.
@@ -235,7 +242,8 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     registerTable(deptNestedTable);
 
     // Register "BONUS" table.
-    MockTable bonusTable = MockTable.create(this, salesSchema, "BONUS", false, 0);
+    MockTable bonusTable =
+        MockTable.create(this, salesSchema, "BONUS", false, 0);
     bonusTable.addColumn("ENAME", varchar20Type);
     bonusTable.addColumn("JOB", varchar10Type);
     bonusTable.addColumn("SAL", intType);
@@ -243,7 +251,8 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     registerTable(bonusTable);
 
     // Register "SALGRADE" table.
-    MockTable salgradeTable = MockTable.create(this, salesSchema, "SALGRADE", false, 5);
+    MockTable salgradeTable =
+        MockTable.create(this, salesSchema, "SALGRADE", false, 5);
     salgradeTable.addColumn("GRADE", intType);
     salgradeTable.addColumn("LOSAL", intType);
     salgradeTable.addColumn("HISAL", intType);
@@ -257,6 +266,20 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     contactAddressTable.addColumn("MAILING_ADDRESS", addressType);
     registerTable(contactAddressTable);
 
+    // Register "DYNAMIC" schema.
+    MockSchema dynamicSchema = new MockSchema("DYNAMIC");
+    registerSchema(dynamicSchema);
+
+    MockTable nationTable =
+        new MockDynamicTable(this, dynamicSchema.getCatalogName(),
+            dynamicSchema.getName(), "NATION", false, 100);
+    registerTable(nationTable);
+
+    MockTable customerTable =
+        new MockDynamicTable(this, dynamicSchema.getCatalogName(),
+            dynamicSchema.getName(), "CUSTOMER", false, 100);
+    registerTable(customerTable);
+
     // Register "CUSTOMER" schema.
     MockSchema customerSchema = new MockSchema("CUSTOMER");
     registerSchema(customerSchema);
@@ -271,6 +294,16 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     contactTable.addColumn("COORD", rectilinearCoordType);
     registerTable(contactTable);
 
+    // Register "CONTACT_PEEK" table. The
+    MockTable contactPeekTable =
+        MockTable.create(this, customerSchema, "CONTACT_PEEK", false, 1000);
+    contactPeekTable.addColumn("CONTACTNO", intType);
+    contactPeekTable.addColumn("FNAME", varchar10Type);
+    contactPeekTable.addColumn("LNAME", varchar10Type);
+    contactPeekTable.addColumn("EMAIL", varchar20Type);
+    contactPeekTable.addColumn("COORD", rectilinearPeekCoordType);
+    registerTable(contactPeekTable);
+
     // Register "ACCOUNT" table.
     MockTable accountTable = MockTable.create(this, customerSchema, "ACCOUNT",
         false, 457);
@@ -428,28 +461,28 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
     MockSchema structTypeSchema = new MockSchema("STRUCT");
     registerSchema(structTypeSchema);
-    MockTable structTypeTable = MockTable.create(
-        this, structTypeSchema, "T", false, 100);
+    MockTable structTypeTable = MockTable.create(this, structTypeSchema, "T",
+        false, 100);
     structTypeTable.addColumn("K0", varchar20Type);
     structTypeTable.addColumn("C1", varchar20Type);
-    FieldInfoBuilder builder = typeFactory.builder();
-    builder.add("C0", intType);
-    builder.add("C1", intType);
-    RelDataType f0Type = new DelegateStructType(
-        typeFactory.createStructType(builder), StructKind.PEEK_FIELDS_DEFAULT);
+    final RelDataType f0Type = typeFactory.builder()
+        .add("C0", intType)
+        .add("C1", intType)
+        .kind(StructKind.PEEK_FIELDS_DEFAULT)
+        .build();
     structTypeTable.addColumn("F0", f0Type);
-    builder = typeFactory.builder();
-    builder.add("C0", intTypeNull);
-    builder.add("C2", intType);
-    builder.add("A0", intType);
-    RelDataType f1Type = new DelegateStructType(
-        typeFactory.createStructType(builder), StructKind.PEEK_FIELDS);
+    final RelDataType f1Type = typeFactory.builder()
+        .add("C0", intTypeNull)
+        .add("C2", intType)
+        .add("A0", intType)
+        .kind(StructKind.PEEK_FIELDS)
+        .build();
     structTypeTable.addColumn("F1", f1Type);
-    builder = typeFactory.builder();
-    builder.add("C3", intType);
-    builder.add("A0", booleanType);
-    RelDataType f2Type = new DelegateStructType(
-        typeFactory.createStructType(builder), StructKind.PEEK_FIELDS);
+    final RelDataType f2Type = typeFactory.builder()
+        .add("C3", intType)
+        .add("A0", booleanType)
+        .kind(StructKind.PEEK_FIELDS)
+        .build();
     structTypeTable.addColumn("F2", f2Type);
     registerTable(structTypeTable);
     return this;
@@ -562,27 +595,17 @@ public class MockCatalogReader implements Prepare.CatalogReader {
   }
 
   public RelDataTypeField field(RelDataType rowType, String alias) {
-    return SqlValidatorUtil.lookupField(caseSensitive, elideRecord, rowType,
-        alias);
-  }
-
-  public int fieldOrdinal(RelDataType rowType, String alias) {
-    final RelDataTypeField field = field(rowType, alias);
-    return field != null ? field.getIndex() : -1;
+    return SqlValidatorUtil.lookupField(caseSensitive, rowType, alias);
   }
 
   public boolean matches(String string, String name) {
     return Util.matches(caseSensitive, string, name);
   }
 
-  public int match(List<String> strings, String name) {
-    return Util.findMatch(strings, name, caseSensitive);
-  }
-
   public RelDataType createTypeFromProjection(final RelDataType type,
       final List<String> columnNameList) {
     return SqlValidatorUtil.createTypeFromProjection(type, columnNameList,
-        typeFactory, caseSensitive, elideRecord);
+        typeFactory, caseSensitive);
   }
 
   private static List<RelCollation> deduceMonotonicity(
@@ -646,6 +669,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     private List<RelCollation> collationList;
     protected final List<String> names;
     private final Set<String> monotonicColumnSet = Sets.newHashSet();
+    private StructKind kind = StructKind.FULLY_QUALIFIED;
 
     public MockTable(MockCatalogReader catalogReader, String catalogName,
         String schemaName, String name, boolean stream, double rowCount) {
@@ -732,7 +756,8 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     }
 
     public void onRegister(RelDataTypeFactory typeFactory) {
-      rowType = typeFactory.createStructType(columnList);
+      rowType = typeFactory.createStructType(kind, Pair.right(columnList),
+          Pair.left(columnList));
       collationList = deduceMonotonicity(this);
     }
 
@@ -771,6 +796,14 @@ public class MockCatalogReader implements Prepare.CatalogReader {
       table.onRegister(catalogReader.typeFactory);
       return table;
     }
+
+    public void setKind(StructKind kind) {
+      this.kind = kind;
+    }
+
+    public StructKind getKind() {
+      return kind;
+    }
   }
 
   /**
@@ -778,13 +811,13 @@ public class MockCatalogReader implements Prepare.CatalogReader {
    * {@link org.apache.calcite.prepare.Prepare.PreparingTable} with dynamic record type.
    */
   public static class MockDynamicTable extends MockTable {
-    public MockDynamicTable(MockCatalogReader catalogReader, String catalogName,
+    MockDynamicTable(MockCatalogReader catalogReader, String catalogName,
         String schemaName, String name, boolean stream, double rowCount) {
       super(catalogReader, catalogName, schemaName, name, stream, rowCount);
     }
 
     public void onRegister(RelDataTypeFactory typeFactory) {
-      rowType =  new DynamicRecordTypeImpl(typeFactory);
+      rowType = new DynamicRecordTypeImpl(typeFactory);
     }
 
     /**
@@ -799,6 +832,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     }
   }
 
+  /** Struct type based on another struct type. */
   private static class DelegateStructType implements RelDataType {
     private RelDataType delegate;
     private StructKind structKind;
@@ -809,92 +843,92 @@ public class MockCatalogReader implements Prepare.CatalogReader {
       this.structKind = structKind;
     }
 
-    @Override public boolean isStruct() {
+    public boolean isStruct() {
       return delegate.isStruct();
     }
 
-    @Override public boolean isDynamicStruct() {
+    public boolean isDynamicStruct() {
       return delegate.isDynamicStruct();
     }
 
-    @Override public List<RelDataTypeField> getFieldList() {
+    public List<RelDataTypeField> getFieldList() {
       return delegate.getFieldList();
     }
 
-    @Override public List<String> getFieldNames() {
+    public List<String> getFieldNames() {
       return delegate.getFieldNames();
     }
 
-    @Override public int getFieldCount() {
+    public int getFieldCount() {
       return delegate.getFieldCount();
     }
 
-    @Override public StructKind getStructKind() {
+    public StructKind getStructKind() {
       return structKind;
     }
 
-    @Override public RelDataTypeField getField(String fieldName,
-        boolean caseSensitive, boolean elideRecord) {
+    public RelDataTypeField getField(String fieldName, boolean caseSensitive,
+        boolean elideRecord) {
       return delegate.getField(fieldName, caseSensitive, elideRecord);
     }
 
-    @Override public boolean isNullable() {
+    public boolean isNullable() {
       return delegate.isNullable();
     }
 
-    @Override public RelDataType getComponentType() {
+    public RelDataType getComponentType() {
       return delegate.getComponentType();
     }
 
-    @Override public RelDataType getKeyType() {
+    public RelDataType getKeyType() {
       return delegate.getKeyType();
     }
 
-    @Override public RelDataType getValueType() {
+    public RelDataType getValueType() {
       return delegate.getValueType();
     }
 
-    @Override public Charset getCharset() {
+    public Charset getCharset() {
       return delegate.getCharset();
     }
 
-    @Override public SqlCollation getCollation() {
+    public SqlCollation getCollation() {
       return delegate.getCollation();
     }
 
-    @Override public SqlIntervalQualifier getIntervalQualifier() {
+    public SqlIntervalQualifier getIntervalQualifier() {
       return delegate.getIntervalQualifier();
     }
 
-    @Override public int getPrecision() {
+    public int getPrecision() {
       return delegate.getPrecision();
     }
 
-    @Override public int getScale() {
+    public int getScale() {
       return delegate.getScale();
     }
 
-    @Override public SqlTypeName getSqlTypeName() {
+    public SqlTypeName getSqlTypeName() {
       return delegate.getSqlTypeName();
     }
 
-    @Override public SqlIdentifier getSqlIdentifier() {
+    public SqlIdentifier getSqlIdentifier() {
       return delegate.getSqlIdentifier();
     }
 
-    @Override public String getFullTypeString() {
+    public String getFullTypeString() {
       return delegate.getFullTypeString();
     }
 
-    @Override public RelDataTypeFamily getFamily() {
+    public RelDataTypeFamily getFamily() {
       return delegate.getFamily();
     }
 
-    @Override public RelDataTypePrecedenceList getPrecedenceList() {
+    public RelDataTypePrecedenceList getPrecedenceList() {
       return delegate.getPrecedenceList();
     }
 
-    @Override public RelDataTypeComparability getComparability() {
+    public RelDataTypeComparability getComparability() {
       return delegate.getComparability();
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/0938c7b6/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 7ec5087..9953355 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -1721,7 +1721,7 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   }
 
   @Test public void testStructType() {
-    sql("select * from struct.t").convertsTo("");
+    sql("select * from struct.t").convertsTo("${plan}");
   }
 
   /**
@@ -1908,6 +1908,14 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
         });
   }
 
+  @Test public void testUnionInFrom() {
+    final String sql = "select x0, x1 from (\n"
+        + "  select 'a' as x0, 'a' as x1, 'a' as x2 from emp\n"
+        + "  union all\n"
+        + "  select 'bb' as x0, 'bb' as x1, 'bb' as x2 from dept)";
+    sql(sql).ok();
+  }
+
   /**
    * Visitor that checks that every {@link RelNode} in a tree is valid.
    *

http://git-wip-us.apache.org/repos/asf/calcite/blob/0938c7b6/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 c48b528..e1afc68 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -4615,9 +4615,10 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     check("select emp.* from emp");
 
     // Error message could be better (EMPNO does exist, but it's a column).
-    checkFails(
-        "select ^empno^ .  * from emp",
-        "Unknown identifier 'EMPNO'");
+    sql("select ^empno^ .  * from emp")
+        .fails("Not a record type. The '\\*' operator requires a record");
+    sql("select ^emp.empno^ .  * from emp")
+        .fails("Not a record type. The '\\*' operator requires a record");
   }
 
   /**
@@ -4636,10 +4637,10 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   @Test public void testNonLocalStar() {
-    // MySQL allows this but we can't, currently
+    // MySQL allows this, and now so do we
     sql("select * from emp e where exists (\n"
-        + "  select ^e^.* from dept where dept.deptno = e.deptno)")
-        .fails("Unknown identifier 'E'");
+        + "  select e.* from dept where dept.deptno = e.deptno)")
+        .type(EMP_RECORD_TYPE);
   }
 
   /**
@@ -4656,10 +4657,9 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   @Test public void testStarDotIdFails() {
-    // Parser allows a star inside (not at end of) compound identifier, but
-    // validator does not
+    // Fails in parser
     sql("select emp.^*^.foo from emp")
-        .fails("Column '\\*' not found in table 'EMP'");
+        .fails("(?s).*Encountered \".\" at .*");
     // Parser does not allow star dot identifier.
     sql("select ^*^.foo from emp")
         .fails("(?s).*Encountered \".\" at .*");
@@ -6950,6 +6950,21 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "DISTINCT/ALL not allowed with COALESCE function");
   }
 
+  @Test public void testColumnNotFound() {
+    sql("select ^b0^ from sales.emp")
+        .fails("Column 'B0' not found in any table");
+  }
+
+  @Test public void testColumnNotFound2() {
+    sql("select ^b0^ from sales.emp, sales.dept")
+        .fails("Column 'B0' not found in any table");
+  }
+
+  @Test public void testColumnNotFound3() {
+    sql("select e.^b0^ from sales.emp as e")
+        .fails("Column 'B0' not found in table 'E'");
+  }
+
   @Test public void testSelectDistinct() {
     check("SELECT DISTINCT deptno FROM emp");
     check("SELECT DISTINCT deptno, sal FROM emp");
@@ -7151,14 +7166,28 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
    * <a href="https://issues.apache.org/jira/browse/CALCITE-497">[CALCITE-497]
    * Support optional qualifier for column name references</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");
+    sql("SELECT contact.^x^, contact.coord.y FROM customer.contact")
+        .fails("Column 'X' not found in table 'CONTACT'");
+
+    // Fully qualified works.
+    sql("SELECT contact.coord.x, contact.coord.y FROM customer.contact")
+        .type("RecordType(INTEGER NOT NULL X, INTEGER NOT NULL Y) NOT NULL");
+
+    // Because the type of CONTACT_PEEK.COORD is marked "peek", the validator
+    // can see through it.
+    sql("SELECT c.x, c.coord.y FROM customer.contact_peek as c")
+        .type("RecordType(INTEGER NOT NULL X, INTEGER NOT NULL Y) NOT NULL");
+    sql("SELECT c.coord.x, c.coord.y FROM customer.contact_peek as c")
+        .type("RecordType(INTEGER NOT NULL X, INTEGER NOT NULL Y) NOT NULL");
+    sql("SELECT x, c.coord.y FROM customer.contact_peek as c")
+        .type("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");
+    final String sql = "SELECT customer.contact_peek.x,\n"
+        + " customer.contact_peek.email, contact_peek.coord.y\n"
+        + "FROM customer.contact_peek";
+    sql(sql).type("RecordType(INTEGER NOT NULL X, VARCHAR(20) NOT NULL EMAIL,"
+        + " INTEGER NOT NULL Y) NOT NULL");
   }
 
   @Test public void testSample() {
@@ -8131,112 +8160,195 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         .fails(cannotStreamResultsForNonStreamingInputs("PRODUCTS, SUPPLIERS"));
   }
 
-  @Test public void testNew() {
+  @Test public void testDummy() {
     // (To debug individual statements, paste them into this method.)
-    //            1         2         3         4         5         6
-    //   12345678901234567890123456789012345678901234567890123456789012345
-    //        check("SELECT count(0) FROM emp GROUP BY ()");
   }
 
   @Test public void testStructType() {
-    // Table STRUCT.T is defined as:
-    // (K0 VARCHAR(20) NOT NULL, C1 VARCHAR(20) NOT NULL,
-    //   RecordType(C0 INTEGER NOT NULL, C1 INTEGER NOT NULL) F0,
-    //   RecordType(C0 INTEGER, C2 INTEGER NOT NULL, A0 INTEGER NOT NULL) F1,
-    //   RecordType(C3 INTEGER NOT NULL, A0 BOOLEAN NOT NULL) F2)
-    // , where F0 has a default struct priority.
+    // Table STRUCT.T is defined as: (
+    //   K0 VARCHAR(20) NOT NULL,
+    //   C1 VARCHAR(20) NOT NULL,
+    //   RecordType:PEEK_FIELDS_DEFAULT(
+    //     C0 INTEGER NOT NULL,
+    //     C1 INTEGER NOT NULL) F0,
+    //   RecordType:PEEK_FIELDS(
+    //      C0 INTEGER,
+    //      C2 INTEGER NOT NULL,
+    //      A0 INTEGER NOT NULL) F1,
+    //   RecordType:PEEK_FIELDS(
+    //      C3 INTEGER NOT NULL,
+    //      A0 BOOLEAN NOT NULL) F2)
+    //
+    // The labels 'PEEK_FIELDS_DEFAULT' and 'PEEK_FIELDS' mean that F0, F1 and
+    // F2 can all be transparent. F0 has default struct priority; F1 and F2 have
+    // lower priority.
 
-    check("select * from struct.t");
+    sql("select * from struct.t").ok();
 
     // Resolve K0 as top-level column K0.
-    checkResultType("select k0 from struct.t",
-        "RecordType(VARCHAR(20) NOT NULL K0) NOT NULL");
+    sql("select k0 from struct.t")
+        .type("RecordType(VARCHAR(20) NOT NULL K0) NOT NULL");
 
     // Resolve C2 as secondary-level column F1.C2.
-    checkResultType("select c2 from struct.t",
-        "RecordType(INTEGER NOT NULL C2) NOT NULL");
+    sql("select c2 from struct.t")
+        .type("RecordType(INTEGER NOT NULL C2) NOT NULL");
 
     // Resolve F1.C2 as fully qualified column F1.C2.
-    checkResultType("select c2 from struct.t",
-        "RecordType(INTEGER NOT NULL C2) NOT NULL");
+    sql("select f1.c2 from struct.t")
+        .type("RecordType(INTEGER NOT NULL C2) NOT NULL");
 
     // Resolve C1 as top-level column C1 as opposed to F0.C1.
-    checkResultType("select c1 from struct.t",
-        "RecordType(VARCHAR(20) NOT NULL C1) NOT NULL");
+    sql("select c1 from struct.t")
+        .type("RecordType(VARCHAR(20) NOT NULL C1) NOT NULL");
 
-    // Resolve C0 as secondary-level column F0.C0 as opposed to F1.C0, since F0 has the
-    // default priority.
-    checkResultType("select c0 from struct.t",
-        "RecordType(INTEGER NOT NULL C0) NOT NULL");
+    // Resolve C0 as secondary-level column F0.C0 as opposed to F1.C0, since F0
+    // has the default priority.
+    sql("select c0 from struct.t")
+        .type("RecordType(INTEGER NOT NULL C0) NOT NULL");
 
-    // Resolve F1.C0 as fully qualified column F1.C0.
-    checkResultType("select f1.c0 from struct.t",
-        "RecordType(INTEGER C0) NOT NULL");
+    // Resolve F1.C0 as fully qualified column F1.C0 (as evidenced by "INTEGER"
+    // rather than "INTEGER NOT NULL")
+    sql("select f1.c0 from struct.t")
+        .type("RecordType(INTEGER C0) NOT NULL");
 
-    // Fail ambiguous column reference A0, since F1.A0 and F2.A0 both exist with the
-    // same resolving priority.
-    checkFails("select ^a0^ from struct.t",
-        "Column 'A0' is ambiguous");
+    // Fail ambiguous column reference A0, since F1.A0 and F2.A0 both exist with
+    // the same resolving priority.
+    sql("select ^a0^ from struct.t")
+        .fails("Column 'A0' is ambiguous");
 
     // Resolve F2.A0 as fully qualified column F2.A0.
-    checkResultType("select f2.a0 from struct.t",
-        "RecordType(BOOLEAN NOT NULL C0) NOT NULL");
+    sql("select f2.a0 from struct.t")
+        .type("RecordType(BOOLEAN NOT NULL A0) NOT NULL");
 
-    // Resolve T0.K0 as top-level column K0, since T0 is recognized as the table alias.
-    checkResultType("select t0.k0 from struct.t t0",
-        "RecordType(VARCHAR(20) NOT NULL K0) NOT NULL");
+    // Resolve T0.K0 as top-level column K0, since T0 is recognized as the table
+    // alias.
+    sql("select t0.k0 from struct.t t0")
+        .type("RecordType(VARCHAR(20) NOT NULL K0) NOT NULL");
+
+    // Resolve T0.C2 as secondary-level column F1.C2, since T0 is recognized as
+    // the table alias here.
+    sql("select t0.c2 from struct.t t0")
+        .type("RecordType(INTEGER NOT NULL C2) NOT NULL");
+
+    // Resolve F0.C2 as secondary-level column F1.C2, since F0 is recognized as
+    // the table alias here.
+    sql("select f0.c2 from struct.t f0")
+        .type("RecordType(INTEGER NOT NULL C2) NOT NULL");
+
+    // Resolve F0.C1 as top-level column C1 as opposed to F0.C1, since F0 is
+    // recognized as the table alias here.
+    sql("select f0.c1 from struct.t f0")
+        .type("RecordType(VARCHAR(20) NOT NULL C1) NOT NULL");
+
+    // Resolve C1 as inner INTEGER column not top-level VARCHAR column.
+    sql("select f0.f0.c1 from struct.t f0")
+        .type("RecordType(INTEGER NOT NULL C1) NOT NULL");
+
+    // Resolve T.C1 as top-level column C1 as opposed to F0.C1, since T is
+    // recognized as the table name.
+    sql("select t.c1 from struct.t")
+        .type("RecordType(VARCHAR(20) NOT NULL C1) NOT NULL");
+
+    // Alias "f0" obscures table name "t"
+    sql("select ^t^.c1 from struct.t f0")
+        .fails("Table 'T' not found");
+
+    // Resolve STRUCT.T.C1 as top-level column C1 as opposed to F0.C1, since
+    // STRUCT.T is recognized as the schema and table name.
+    sql("select struct.t.c1 from struct.t")
+        .type("RecordType(VARCHAR(20) NOT NULL C1) NOT NULL");
+
+    // Table alias "f0" obscures table name "struct.t"
+    sql("select ^struct.t^.c1 from struct.t f0")
+        .fails("Table 'STRUCT.T' not found");
+
+    // Resolve F0.F0.C1 as secondary-level column F0.C1, since the first F0 is
+    // recognized as the table alias here.
+    sql("select f0.f0.c1 from struct.t f0")
+        .type("RecordType(INTEGER NOT NULL C1) NOT NULL");
+
+    // Resolve T.F0.C1 as secondary-level column F0.C1, since T is recognized as
+    // the table name.
+    sql("select t.f0.c1 from struct.t")
+        .type("RecordType(INTEGER NOT NULL C1) NOT NULL");
 
-    // Resolve T0.C2 as secondary-level column F1.C2, since T0 is recognized as the
-    // table alias here.
-    checkResultType("select t0.c2 from struct.t t0",
-        "RecordType(INTEGER NOT NULL C2) NOT NULL");
+    // Table alias obscures
+    sql("select ^t.f0^.c1 from struct.t f0")
+        .fails("Table 'T.F0' not found");
 
-    // Resolve F0.C2 as secondary-level column F1.C2, since F0 is recognized as the
-    // table alias here.
-    checkResultType("select f0.c2 from struct.t f0",
-        "RecordType(INTEGER NOT NULL C2) NOT NULL");
+    // Resolve STRUCT.T.F0.C1 as secondary-level column F0.C1, since STRUCT.T is
+    // recognized as the schema and table name.
+    sql("select struct.t.f0.c1 from struct.t")
+        .type("RecordType(INTEGER NOT NULL C1) NOT NULL");
 
-    // Resolve F0.C1 as top-level column C1 as opposed to F0.C1, since F0 is recognized
-    // as the table alias here.
-    checkResultType("select f0.c1 from struct.t f0",
-        "RecordType(VARCHAR(20) NOT NULL C1) NOT NULL");
+    // Table alias "f0" obscures table name "struct.t"
+    sql("select ^struct.t.f0^.c1 from struct.t f0")
+        .fails("Table 'STRUCT.T.F0' not found");
 
-    // Resolve T.C1 as top-level column C1 as opposed to F0.C1, since T is recognized as
-    // the table name.
-    checkResultType("select t.c1 from struct.t f0",
-        "RecordType(VARCHAR(20) NOT NULL C1) NOT NULL");
+    // Resolve struct type F1 with wildcard.
+    sql("select f1.* from struct.t")
+        .type("RecordType(INTEGER C0, INTEGER NOT NULL C2,"
+            + " INTEGER NOT NULL A0) NOT NULL");
 
-    // Resolve STRUCT.T.C1 as top-level column C1 as opposed to F0.C1, since STRUCT.T is
-    // recognized as the schema and table name.
-    checkResultType("select struct.t.c1 from struct.t f0",
-        "RecordType(VARCHAR(20) NOT NULL C1) NOT NULL");
+    // Fail non-existent column B0.
+    sql("select ^b0^ from struct.t")
+        .fails("Column 'B0' not found in any table");
 
-    // Resolve F0.F0.C1 as secondary-level column F0.C1, since the first F0 is recognized
-    // as the table alias here.
-    checkResultType("select f0.f0.c1 from struct.t f0",
-        "RecordType(INTEGER NOT NULL C1) NOT NULL");
+    // It's OK to reference a record type.
+    //
+    // This is admittedly a bit strange for Phoenix users. We model a column
+    // family as a column whose type is a record, but Phoenix users would
+    // rarely if ever want to use a column family as a record.
+    sql("select f1 from struct.t")
+        .type("RecordType(RecordType:peek(INTEGER C0, INTEGER NOT NULL C2,"
+            + " INTEGER NOT NULL A0) NOT NULL F1) NOT NULL");
+
+    // If we fail to find a column, give an error based on the shortest prefix
+    // that fails.
+    sql("select t.^f0.notFound^.a.b.c.d from struct.t")
+        .fails("Column 'F0\\.NOTFOUND' not found in table 'T'");
+    sql("select t.^f0.notFound^ from struct.t")
+        .fails("Column 'F0\\.NOTFOUND' not found in table 'T'");
+    sql("select t.^f0.c1.notFound^ from struct.t")
+        .fails("Column 'F0\\.C1\\.NOTFOUND' not found in table 'T'");
+  }
 
-    // Resolve T.F0.C1 as secondary-level column F0.C1, since T is recognized as the table
-    // name.
-    checkResultType("select t.f0.c1 from struct.t f0",
-        "RecordType(INTEGER NOT NULL C1) NOT NULL");
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]
+   * Dynamic Table / Dynamic Star support</a>. */
+  @Test public void testAmbiguousDynamicStar() throws Exception {
+    final String sql = "select ^n_nation^\n"
+        + "from (select * from \"DYNAMIC\".NATION),\n"
+        + " (select * from \"DYNAMIC\".CUSTOMER)";
+    sql(sql).fails("Column 'N_NATION' is ambiguous");
+  }
 
-    // Resolve STRUCT.T.F0.C1 as secondary-level column F0.C1, since STRUCT.T is
-    // recognized as the schema and table name.
-    checkResultType("select struct.t.f0.c1 from struct.t f0",
-        "RecordType(INTEGER NOT NULL C1) NOT NULL");
+  @Test public void testAmbiguousDynamicStar2() throws Exception {
+    final String sql = "select ^n_nation^\n"
+        + "from (select * from \"DYNAMIC\".NATION, \"DYNAMIC\".CUSTOMER)";
+    sql(sql).fails("Column 'N_NATION' is ambiguous");
+  }
 
-    // Resolve struct type F1 with wildcard.
-    checkResultType("select f1.* from struct.t",
-        "RecordType(INTEGER NOT NULL C0, INTEGER NOT NULL C2) NOT NULL");
+  @Test public void testAmbiguousDynamicStar3() throws Exception {
+    final String sql = "select ^nc.n_nation^\n"
+        + "from (select * from \"DYNAMIC\".NATION, \"DYNAMIC\".CUSTOMER) as nc";
+    sql(sql).fails("Column 'N_NATION' is ambiguous");
+  }
 
-    // Fail non-existent column B0.
-    checkFails("select ^b0^ from struct.t",
-        "Column 'B0' not found in any table");
+  @Test public void testAmbiguousDynamicStar4() throws Exception {
+    final String sql = "select n.n_nation\n"
+        + "from (select * from \"DYNAMIC\".NATION) as n,\n"
+        + " (select * from \"DYNAMIC\".CUSTOMER)";
+    sql(sql).type("RecordType(ANY N_NATION) NOT NULL");
+  }
 
-    // Fail struct type with no wildcard.
-    checkFails("select ^f1^ from struct.t",
-        "Unknown identifier 'F1'");
+  /** When resolve column reference, regular field has higher priority than
+   * dynamic star columns. */
+  @Test public void testDynamicStar2() throws Exception {
+    final String sql = "select newid from (\n"
+        + "  select *, NATION.N_NATION + 100 as newid\n"
+        + "  from \"DYNAMIC\".NATION, \"DYNAMIC\".CUSTOMER)";
+    sql(sql).type("RecordType(ANY NEWID) NOT NULL");
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/0938c7b6/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 6713fd4..3c29e67 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -2469,6 +2469,19 @@ LogicalTableModify(table=[[CATALOG, SALES, EMP]], operation=[INSERT], updateColu
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testStructType">
+        <Resource name="sql">
+            <![CDATA[select * from struct.t]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(K0=[$0], C1=[$1], F0=[$2], F1=[$3], F2=[$4])
+  LogicalProject(K0=[$0], C1=[$1], F0=[$2], F03=[$3], F1=[$4], F15=[$5], F16=[$6], F2=[$7], F28=[$8])
+    LogicalProject(K0=[$0], C1=[$1], C0=[$2.C0], C13=[$2.C1], C04=[$3.C0], C2=[$3.C2], A0=[$3.A0], C3=[$4.C3], A08=[$4.A0])
+      LogicalTableScan(table=[[CATALOG, STRUCT, T]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testWindowAggWithGroupBy">
         <Resource name="sql">
             <![CDATA[select min(deptno), rank() over (order by empno),
@@ -3073,6 +3086,24 @@ LogicalAggregate(group=[{}], EXPR$0=[MIN($0)])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testUnionInFrom">
+        <Resource name="sql">
+            <![CDATA[select x0, x1 from (
+  select 'a' as x0, 'a' as x1, 'a' as x2 from emp
+  union all
+  select 'bb' as x0, 'bb' as x1, 'bb' as x2 from dept)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(X0=[$0], X1=[$1])
+  LogicalUnion(all=[true])
+    LogicalProject(X0=['a'], X1=['a'], X2=['a'])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(X0=['bb'], X1=['bb'], X2=['bb'])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testWhereInCorrelated">
         <Resource name="sql">
             <![CDATA[select empno from emp as e

http://git-wip-us.apache.org/repos/asf/calcite/blob/0938c7b6/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkPushDownRule.java
----------------------------------------------------------------------
diff --git a/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkPushDownRule.java b/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkPushDownRule.java
index 31e73d8..fdf5b9d 100644
--- a/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkPushDownRule.java
+++ b/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkPushDownRule.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.adapter.splunk;
 
 import org.apache.calcite.adapter.splunk.util.StringUtils;
+import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
@@ -25,7 +26,6 @@ import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rel.type.RelRecordType;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
@@ -189,6 +189,7 @@ public class SplunkPushDownRule
       LogicalProject bottomProj,
       RelDataType topRow,
       RelDataType bottomRow) {
+    final RelOptCluster cluster = splunkRel.getCluster();
     StringBuilder updateSearchStr = new StringBuilder(splunkRel.search);
 
     if (!toAppend.isEmpty()) {
@@ -237,7 +238,7 @@ public class SplunkPushDownRule
         if (!bottomFields.get(rif.getIndex()).getName()
             .equals(topFields.get(i).getName())) {
           renames.add(
-              new Pair<String, String>(
+              Pair.of(
                   bottomFields.get(rif.getIndex()).getName(),
                   topFields.get(i).getName()));
           field = topFields.get(i);
@@ -254,12 +255,13 @@ public class SplunkPushDownRule
       }
     }
 
-    RelDataType resultType = new RelRecordType(newFields);
+    RelDataType resultType =
+        cluster.getTypeFactory().createStructType(newFields);
     String searchWithFilter = updateSearchStr.toString();
 
     RelNode rel =
         new SplunkTableScan(
-            splunkRel.getCluster(),
+            cluster,
             splunkRel.getTable(),
             splunkRel.splunkTable,
             searchWithFilter,
@@ -267,8 +269,7 @@ public class SplunkPushDownRule
             splunkRel.latest,
             resultType.getFieldNames());
 
-    LOGGER.debug(
-        "end of appendSearchString fieldNames: {}",
+    LOGGER.debug("end of appendSearchString fieldNames: {}",
         rel.getRowType().getFieldNames());
     return rel;
   }