You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jn...@apache.org on 2016/05/18 05:05:36 UTC

calcite git commit: [CALCITE-1150] Add dynamic record type and dynamic star for schema-on-read table

Repository: calcite
Updated Branches:
  refs/heads/master 720b117ed -> 9bd7d7550


[CALCITE-1150] Add dynamic record type and dynamic star for schema-on-read table

 * Add dynamic star column and dynamic record type. Use "**" as the dynamic star column name prefix,
 * RelOptTableImpl.toRel() will convert a dynamicRecordType to RelRecordType. This ensures an immutable record type in sql-to-rel,
 * Add RelDataType.isDynamicStruct().

Close apache/calcite/#228


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

Branch: refs/heads/master
Commit: 9bd7d7550cfe31b76e3970a532014a8186e1ff5a
Parents: 720b117
Author: Jinfeng Ni <jn...@apache.org>
Authored: Wed Mar 9 18:49:01 2016 -0800
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Tue May 17 21:56:38 2016 -0700

----------------------------------------------------------------------
 .../apache/calcite/prepare/RelOptTableImpl.java |  17 ++
 .../calcite/rel/type/DynamicRecordType.java     |  42 +++++
 .../calcite/rel/type/DynamicRecordTypeImpl.java |  91 ++++++++++
 .../apache/calcite/rel/type/RelDataType.java    |   5 +
 .../calcite/rel/type/RelDataTypeField.java      |   5 +
 .../calcite/rel/type/RelDataTypeFieldImpl.java  |   7 +
 .../calcite/rel/type/RelDataTypeHolder.java     |  93 +++++++++++
 .../calcite/rel/type/RelDataTypeImpl.java       |  13 ++
 .../org/apache/calcite/sql/SqlIdentifier.java   |   6 +
 .../apache/calcite/sql/fun/SqlItemOperator.java |   2 +
 .../apache/calcite/sql/type/SqlTypeName.java    |   4 +-
 .../calcite/sql/validate/DelegatingScope.java   |  40 ++++-
 .../calcite/sql/validate/OrderByScope.java      |   5 +-
 .../calcite/sql/validate/SqlValidatorImpl.java  | 139 +++++++++++----
 .../calcite/sql/validate/SqlValidatorUtil.java  |   7 +
 .../calcite/jdbc/CalciteRemoteDriverTest.java   |   2 +-
 .../apache/calcite/test/MockCatalogReader.java  |  41 ++++-
 .../calcite/test/SqlToRelConverterTest.java     | 167 +++++++++++++++++++
 .../apache/calcite/test/SqlToRelTestBase.java   |   2 +-
 .../apache/calcite/test/SqlValidatorTest.java   |  22 +--
 .../calcite/test/SqlToRelConverterTest.xml      | 138 +++++++++++++++
 21 files changed, 797 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
index e3ce221..05c1127 100644
--- a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
@@ -31,6 +31,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.logical.LogicalTableScan;
 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.schema.ExtensibleTable;
 import org.apache.calcite.schema.FilterableTable;
 import org.apache.calcite.schema.ModifiableTable;
@@ -124,6 +125,14 @@ public class RelOptTableImpl implements Prepare.PreparingTable {
         table, expressionFunction, rowCount);
   }
 
+  /**
+   * Creates a copy of this RelOptTable. The new RelOptTable will have newRowType.
+   */
+  public RelOptTableImpl copy(RelDataType newRowType) {
+    return new RelOptTableImpl(this.schema, newRowType, this.names, this.table,
+        this.expressionFunction, this.rowCount);
+  }
+
   private static Function<Class, Expression> getClassExpressionFunction(
       CalciteSchema.TableEntry tableEntry, Table table) {
     return getClassExpressionFunction(tableEntry.schema.plus(), tableEntry.name,
@@ -233,6 +242,14 @@ public class RelOptTableImpl implements Prepare.PreparingTable {
   }
 
   public RelNode toRel(ToRelContext context) {
+    // Make sure rowType's list is immutable. If rowType is DynamicRecordType, creates a new
+    // RelOptTable by replacing with immutable RelRecordType using the same field list.
+    if (this.getRowType().isDynamicStruct()) {
+      final RelDataType staticRowType = new RelRecordType(getRowType().getFieldList());
+      final RelOptTable relOptTable = this.copy(staticRowType);
+      return relOptTable.toRel(context);
+    }
+
     if (table instanceof TranslatableTable) {
       return ((TranslatableTable) table).toRel(context, this);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/main/java/org/apache/calcite/rel/type/DynamicRecordType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/DynamicRecordType.java b/core/src/main/java/org/apache/calcite/rel/type/DynamicRecordType.java
new file mode 100644
index 0000000..642a760
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/type/DynamicRecordType.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.rel.type;
+
+
+/**
+ * Specific type of RelRecordType that corresponds to a dynamic table,
+ * where columns are created as they are requested.
+ */
+public abstract class DynamicRecordType extends RelDataTypeImpl {
+
+  // The prefix string for dynamic star column name
+  public static final String DYNAMIC_STAR_PREFIX = "**";
+
+  public boolean isDynamicStruct() {
+    return true;
+  }
+
+  /**
+   * Returns true if the column name starts with DYNAMIC_STAR_PREFIX.
+   */
+  public static boolean isDynamicStarColName(String name) {
+    return name.startsWith(DYNAMIC_STAR_PREFIX);
+  }
+
+}
+
+// End DynamicRecordType.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/main/java/org/apache/calcite/rel/type/DynamicRecordTypeImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/DynamicRecordTypeImpl.java b/core/src/main/java/org/apache/calcite/rel/type/DynamicRecordTypeImpl.java
new file mode 100644
index 0000000..51e8c98
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/type/DynamicRecordTypeImpl.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.calcite.rel.type;
+
+import org.apache.calcite.sql.type.SqlTypeExplicitPrecedenceList;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Pair;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Implementation of RelDataType for dynamic table. It's used in
+ * Sql validation phase, where field list is mutable for getField() call.
+ *
+ * <p>After Sql validation, a normal RelDataTypeImpl with immutable field list
+ * would take place of DynamicRecordTypeImpl instance for dynamic table. </p>
+ */
+public class DynamicRecordTypeImpl extends DynamicRecordType {
+
+  private final RelDataTypeFactory typeFactory;
+  private final RelDataTypeHolder holder;
+
+  public DynamicRecordTypeImpl(RelDataTypeFactory typeFactory) {
+    this.typeFactory = typeFactory;
+    this.holder = new RelDataTypeHolder();
+    this.holder.setRelDataTypeFactory(typeFactory);
+    computeDigest();
+  }
+
+  public List<RelDataTypeField> getFieldList() {
+    return holder.getFieldList(typeFactory);
+  }
+
+  public int getFieldCount() {
+    return holder.getFieldCount();
+  }
+
+  public RelDataTypeField getField(String fieldName, boolean caseSensitive, boolean elideRecord) {
+    Pair<RelDataTypeField, Boolean> pair = holder.getFieldOrInsert(typeFactory, fieldName,
+        caseSensitive);
+    // If a new field is added, we should re-compute the digest.
+    if (pair.right) {
+      computeDigest();
+    }
+
+    return pair.left;
+  }
+
+  public List<String> getFieldNames() {
+    return holder.getFieldNames();
+  }
+
+  public SqlTypeName getSqlTypeName() {
+    return SqlTypeName.ROW;
+  }
+
+  public RelDataTypePrecedenceList getPrecedenceList() {
+    return new SqlTypeExplicitPrecedenceList(Collections.<SqlTypeName>emptyList());
+  }
+
+  protected void generateTypeString(StringBuilder sb, boolean withDetail) {
+    sb.append("(DynamicRecordRow" + getFieldNames() + ")");
+  }
+
+  public boolean isStruct() {
+    return true;
+  }
+
+  public RelDataTypeFamily getFamily() {
+    return getSqlTypeName().getFamily();
+  }
+
+}
+
+// End DynamicRecordTypeImpl.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/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 0133470..9bd73f2 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
@@ -227,6 +227,11 @@ public interface RelDataType /*extends Type*/ {
    * applied to values of this type
    */
   RelDataTypeComparability getComparability();
+
+  /**
+   *@return whether it has dynamic structure (for "schema-on-read" table)
+   */
+  boolean isDynamicStruct();
 }
 
 // End RelDataType.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeField.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeField.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeField.java
index 2094d48..293de27 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeField.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeField.java
@@ -50,6 +50,11 @@ public interface RelDataTypeField extends Map.Entry<String, RelDataType> {
    * @return field type
    */
   RelDataType getType();
+
+  /**
+   * Returns true if this is a dynamic star field.
+   */
+  boolean isDynamicStar();
 }
 
 // End RelDataTypeField.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFieldImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFieldImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFieldImpl.java
index 59eea0c..a3cbf32 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFieldImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFieldImpl.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.rel.type;
 
+import org.apache.calcite.sql.type.SqlTypeName;
+
 import java.io.Serializable;
 
 /**
@@ -99,6 +101,11 @@ public class RelDataTypeFieldImpl implements RelDataTypeField, Serializable {
   public String toString() {
     return "#" + index + ": " + name + " " + type;
   }
+
+  public boolean isDynamicStar() {
+    return type.getSqlTypeName() == SqlTypeName.DYNAMIC_STAR;
+  }
+
 }
 
 // End RelDataTypeFieldImpl.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeHolder.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeHolder.java
new file mode 100644
index 0000000..6db8519
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeHolder.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.calcite.rel.type;
+
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Holding the expandable list of fields for dynamic table.
+ */
+public class RelDataTypeHolder {
+  List<RelDataTypeField> fields = new ArrayList<>();
+
+  private RelDataTypeFactory typeFactory;
+
+  public List<RelDataTypeField> getFieldList(RelDataTypeFactory typeFactory) {
+    return fields;
+  }
+
+  public int getFieldCount() {
+    return fields.size();
+  }
+
+  /**
+   * Get field if exists, otherwise inserts a new field. The new field by default will have "any"
+   * type, except for the dynamic star field.
+   *
+   * @param typeFactory RelDataTypeFactory
+   * @param fieldName Request field name
+   * @param caseSensitive Case Sensitive
+   * @return A pair of RelDataTypeField and Boolean. Boolean indicates whether a new field is added
+   * to this holder.
+   */
+  public Pair<RelDataTypeField, Boolean> getFieldOrInsert(RelDataTypeFactory typeFactory,
+      String fieldName, boolean caseSensitive) {
+
+    // First check if this field name exists in our field list
+    for (RelDataTypeField f : fields) {
+      if (Util.matches(caseSensitive, f.getName(), fieldName)) {
+        return Pair.of(f, false);
+      }
+    }
+
+    final SqlTypeName typeName = DynamicRecordType.isDynamicStarColName(fieldName)
+        ? SqlTypeName.DYNAMIC_STAR : SqlTypeName.ANY;
+
+    // This field does not exist in our field list add it
+    RelDataTypeField newField = new RelDataTypeFieldImpl(
+        fieldName,
+        fields.size(),
+        typeFactory.createTypeWithNullability(typeFactory.createSqlType(typeName), true));
+
+    // Add the name to our list of field names
+    fields.add(newField);
+
+    return Pair.of(newField, true);
+  }
+
+  public List<String> getFieldNames() {
+    List<String> fieldNames = new ArrayList<>();
+    for (RelDataTypeField f : fields) {
+      fieldNames.add(f.getName());
+    }
+
+    return fieldNames;
+  }
+
+  public void setRelDataTypeFactory(RelDataTypeFactory typeFactory) {
+    this.typeFactory = typeFactory;
+  }
+
+}
+
+// End RelDataTypeHolder.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/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 0c1335b..3134b15 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
@@ -107,6 +107,15 @@ public abstract class RelDataTypeImpl
             fieldName, -1, lastField.getType());
       }
     }
+
+    // a dynamic * field will match any field name.
+    for (RelDataTypeField field : fieldList) {
+      if (field.isDynamicStar()) {
+        // the requested field could be in the unresolved star
+        return field;
+      }
+    }
+
     return null;
   }
 
@@ -390,6 +399,10 @@ public abstract class RelDataTypeImpl
     return rowType.getField("_extra", true, false);
   }
 
+  public boolean isDynamicStruct() {
+    return false;
+  }
+
   /** Work space for {@link RelDataTypeImpl#getFieldRecurse}. */
   private static class Slot {
     int count;

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java b/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
index 2a6bca0..d9b44ba 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql;
 
+import org.apache.calcite.rel.type.DynamicRecordType;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
@@ -332,6 +333,11 @@ public class SqlIdentifier extends SqlNode {
   }
 
   public SqlMonotonicity getMonotonicity(SqlValidatorScope scope) {
+    // for "star" column, whether it's static or dynamic return not_monotonic directly.
+    if (Util.last(names).equals("") || DynamicRecordType.isDynamicStarColName(Util.last(names))) {
+      return SqlMonotonicity.NOT_MONOTONIC;
+    }
+
     // First check for builtin functions which don't have parentheses,
     // like "LOCALTIME".
     final SqlValidator validator = scope.getValidator();

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/main/java/org/apache/calcite/sql/fun/SqlItemOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlItemOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlItemOperator.java
index f73ce4a..6b07a35 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlItemOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlItemOperator.java
@@ -104,6 +104,7 @@ class SqlItemOperator extends SqlSpecialOperator {
       return OperandTypes.family(
           operandType.getKeyType().getSqlTypeName().getFamily());
     case ANY:
+    case DYNAMIC_STAR:
       return OperandTypes.or(
           OperandTypes.family(SqlTypeFamily.INTEGER),
           OperandTypes.family(SqlTypeFamily.CHARACTER));
@@ -128,6 +129,7 @@ class SqlItemOperator extends SqlSpecialOperator {
       return typeFactory.createTypeWithNullability(operandType.getValueType(),
           true);
     case ANY:
+    case DYNAMIC_STAR:
       return typeFactory.createTypeWithNullability(
           typeFactory.createSqlType(SqlTypeName.ANY), true);
     default:

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
index 7d0c590..b43850c 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
@@ -85,7 +85,9 @@ public enum SqlTypeName {
   CURSOR(PrecScale.NO_NO, false, ExtraSqlTypes.REF_CURSOR,
       SqlTypeFamily.CURSOR),
   COLUMN_LIST(PrecScale.NO_NO, false, Types.OTHER + 2,
-      SqlTypeFamily.COLUMN_LIST);
+      SqlTypeFamily.COLUMN_LIST),
+  DYNAMIC_STAR(PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES, true,
+      Types.JAVA_OBJECT, SqlTypeFamily.ANY);
 
   public static final int MAX_DATETIME_PRECISION = 3;
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
index d5049e7..383fd12 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql.validate;
 
+import org.apache.calcite.rel.type.DynamicRecordType;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.SqlCall;
@@ -149,25 +150,31 @@ public abstract class DelegatingScope implements SqlValidatorScope {
 
     String columnName;
     switch (identifier.names.size()) {
-    case 1:
+    case 1: {
       columnName = identifier.names.get(0);
       final Pair<String, SqlValidatorNamespace> pair =
           findQualifyingTableName(columnName, identifier);
       final String tableName = pair.left;
       final SqlValidatorNamespace namespace = pair.right;
 
+      final RelDataTypeField field =
+          validator.catalogReader.field(namespace.getRowType(), columnName);
+
+      checkAmbiguousUnresolvedStar(namespace.getRowType(), field, identifier, columnName);
+
       // todo: do implicit collation here
       final SqlParserPos pos = identifier.getParserPosition();
       SqlIdentifier expanded =
           new SqlIdentifier(
-              ImmutableList.of(tableName, columnName),
+              ImmutableList.of(tableName, field.getName()),  // use resolved field name
               null,
               pos,
               ImmutableList.of(SqlParserPos.ZERO, pos));
       validator.setOriginal(expanded, identifier);
       return SqlQualified.create(this, 1, namespace, expanded);
+    }
 
-    default:
+    default: {
       SqlValidatorNamespace fromNs = null;
       final int size = identifier.names.size();
       int i = size - 1;
@@ -194,6 +201,9 @@ public abstract class DelegatingScope implements SqlValidatorScope {
               RESOURCE.columnNotFoundInTable(columnName,
                   identifier.getComponent(0, j).toString()));
         }
+
+        checkAmbiguousUnresolvedStar(fromRowType, field, identifier, columnName);
+
         // normalize case to match definition, in a copy of the identifier
         identifier = identifier.setName(j, field.getName());
         fromRowType = field.getType();
@@ -210,6 +220,7 @@ public abstract class DelegatingScope implements SqlValidatorScope {
       }
       return SqlQualified.create(this, i, fromNs, identifier);
     }
+    }
   }
 
   public void validateExpr(SqlNode expr) {
@@ -229,6 +240,29 @@ public abstract class DelegatingScope implements SqlValidatorScope {
     return parent.getOrderList();
   }
 
+  private void checkAmbiguousUnresolvedStar(RelDataType fromRowType, RelDataTypeField field,
+      SqlIdentifier identifier, String columnName) {
+
+    if (field != null
+        && field.isDynamicStar()
+        && !DynamicRecordType.isDynamicStarColName(columnName)) {
+      // Make sure fromRowType only contains one star column.
+      // Having more than one star columns implies ambiguous column.
+      int count = 0;
+      for (RelDataTypeField possibleStar : fromRowType.getFieldList()) {
+        if (possibleStar.isDynamicStar()) {
+          count++;
+        }
+      }
+
+      if (count > 1) {
+        throw validator.newValidationError(identifier,
+            RESOURCE.columnAmbiguous(columnName));
+      }
+    }
+
+  }
+
   /**
    * Returns the parent scope of this <code>DelegatingScope</code>.
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java b/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java
index 55b3af9..65958fd 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java
@@ -73,7 +73,10 @@ public class OrderByScope extends DelegatingScope {
       final SqlValidatorNamespace selectNs =
           validator.getNamespace(select);
       final RelDataType rowType = selectNs.getRowType();
-      if (validator.catalogReader.field(rowType, name) != null) {
+
+      final RelDataTypeField field = validator.catalogReader.field(rowType, name);
+      if (field != null && !field.isDynamicStar()) {
+        // if identifier is resolved to a dynamic star, use super.fullyQualify() for such case.
         return SqlQualified.create(this, 1, selectNs, identifier);
       }
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index d2ad4c7..df95dd6 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -19,6 +19,7 @@ package org.apache.calcite.sql.validate;
 import org.apache.calcite.config.NullCollation;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.type.DynamicRecordType;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -453,30 +454,48 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     if (!identifier.isStar()) {
       return false;
     }
-    final SqlParserPos starPosition = identifier.getParserPosition();
+    final SqlParserPos startPosition = identifier.getParserPosition();
     switch (identifier.names.size()) {
     case 1:
       for (Pair<String, SqlValidatorNamespace> p : scope.children) {
-        final SqlNode from = p.right.getNode();
-        final SqlValidatorNamespace fromNs = getNamespace(from, scope);
-        assert fromNs != null;
-        final RelDataType rowType = fromNs.getRowType();
-        for (RelDataTypeField field : rowType.getFieldList()) {
-          String columnName = field.getName();
 
-          // TODO: do real implicit collation here
-          final SqlNode exp =
-              new SqlIdentifier(
-                  ImmutableList.of(p.left, columnName),
-                  starPosition);
+        if (p.right.getRowType().isDynamicStruct()) {
+          // don't expand star if the underneath table is dynamic.
+          // Treat this star as a special field in validation/conversion and
+          // wait until execution time to expand this star.
+          final SqlNode exp = new SqlIdentifier(
+                  ImmutableList.of(p.left, DynamicRecordType.DYNAMIC_STAR_PREFIX),
+                  startPosition);
           addToSelectList(
-              selectItems,
-              aliases,
-              types,
-              exp,
-              scope,
-              includeSystemVars);
+               selectItems,
+               aliases,
+               types,
+               exp,
+               scope,
+               includeSystemVars);
+        } else {
+          final SqlNode from = p.right.getNode();
+          final SqlValidatorNamespace fromNs = getNamespace(from, scope);
+          assert fromNs != null;
+          final RelDataType rowType = fromNs.getRowType();
+          for (RelDataTypeField field : rowType.getFieldList()) {
+            String columnName = field.getName();
+
+            // TODO: do real implicit collation here
+            final SqlNode exp =
+                new SqlIdentifier(
+                    ImmutableList.of(p.left, columnName),
+                    startPosition);
+            addToSelectList(
+                selectItems,
+                aliases,
+                types,
+                exp,
+                scope,
+                includeSystemVars);
+          }
         }
+
       }
       return true;
     default:
@@ -502,18 +521,31 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       }
       assert fromNs != null;
       final RelDataType rowType = fromNs.getRowType();
-      for (RelDataTypeField field : rowType.getFieldList()) {
-        String columnName = field.getName();
 
-        // TODO: do real implicit collation here
+      if (rowType.isDynamicStruct()) {
+        // don't expand star if the underneath table is dynamic.
         addToSelectList(
             selectItems,
             aliases,
             types,
-            prefixId.plus(columnName, starPosition),
+            prefixId.plus(DynamicRecordType.DYNAMIC_STAR_PREFIX, startPosition),
             scope,
             includeSystemVars);
+      } else {
+        for (RelDataTypeField field : rowType.getFieldList()) {
+          String columnName = field.getName();
+
+          // TODO: do real implicit collation here
+          addToSelectList(
+              selectItems,
+              aliases,
+              types,
+              prefixId.plus(columnName, startPosition),
+              scope,
+              includeSystemVars);
+        }
       }
+
       return true;
     }
   }
@@ -985,7 +1017,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
         // a half-hearted resolution now in case it's a
         // builtin function requiring special casing.  If it's
         // not, we'll handle it later during overload resolution.
-        final List<SqlOperator> overloads = Lists.newArrayList();
+        final List<SqlOperator> overloads = new ArrayList<>();
         opTab.lookupOperatorOverloads(function.getNameAsId(),
             function.getFunctionType(), SqlSyntax.FUNCTION, overloads);
         if (overloads.size() == 1) {
@@ -1521,7 +1553,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       SqlFunction unresolvedFunction, List<RelDataType> argTypes,
       List<String> argNames) {
     // For builtins, we can give a better error message
-    final List<SqlOperator> overloads = Lists.newArrayList();
+    final List<SqlOperator> overloads = new ArrayList<>();
     opTab.lookupOperatorOverloads(unresolvedFunction.getNameAsId(), null,
         SqlSyntax.FUNCTION, overloads);
     if (overloads.size() == 1) {
@@ -2786,6 +2818,11 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       break;
     case ON:
       Util.permAssert(condition != null, "condition != null");
+      if (condition != null) {
+        SqlNode expandedCondition = expand(condition, joinScope);
+        join.setOperand(5, expandedCondition);
+        condition = join.getCondition();
+      }
       validateWhereOrOn(joinScope, condition, "ON");
       break;
     case USING:
@@ -3265,7 +3302,19 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     final SqlValidatorScope orderScope = getOrderScope(select);
 
     Util.permAssert(orderScope != null, "orderScope != null");
+
+    List<SqlNode> expandList = new ArrayList<>();
     for (SqlNode orderItem : orderList) {
+      SqlNode expandedOrderItem = expand(orderItem, orderScope);
+      expandList.add(expandedOrderItem);
+    }
+
+    SqlNodeList expandedOrderList = new SqlNodeList(
+        expandList,
+        orderList.getParserPosition());
+    select.setOrderBy(expandedOrderList);
+
+    for (SqlNode orderItem : expandedOrderList) {
       validateOrderItem(select, orderItem);
     }
   }
@@ -3302,6 +3351,15 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     final SqlValidatorScope groupScope = getGroupScope(select);
     inferUnknownTypes(unknownType, groupScope, groupList);
 
+    // expand the expression in group list.
+    List<SqlNode> expandedList = new ArrayList<>();
+    for (SqlNode groupItem : groupList) {
+      SqlNode expandedItem = expand(groupItem, groupScope);
+      expandedList.add(expandedItem);
+    }
+    groupList = new SqlNodeList(expandedList, groupList.getParserPosition());
+    select.setGroupBy(groupList);
+
     // Nodes in the GROUP BY clause are expressions except if they are calls
     // to the GROUPING SETS, ROLLUP or CUBE operators; this operators are not
     // expressions, because they do not have a type.
@@ -3371,7 +3429,9 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       return;
     }
     final SqlValidatorScope whereScope = getWhereScope(select);
-    validateWhereOrOn(whereScope, where, "WHERE");
+    final SqlNode expandedWhere = expand(where, whereScope);
+    select.setWhere(expandedWhere);
+    validateWhereOrOn(whereScope, expandedWhere, "WHERE");
   }
 
   protected void validateWhereOrOn(
@@ -3422,9 +3482,9 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
 
     // Validate SELECT list. Expand terms of the form "*" or "TABLE.*".
     final SqlValidatorScope selectScope = getSelectScope(select);
-    final List<SqlNode> expandedSelectItems = Lists.newArrayList();
+    final List<SqlNode> expandedSelectItems = new ArrayList<>();
     final Set<String> aliases = Sets.newHashSet();
-    final List<Map.Entry<String, RelDataType>> fieldList = Lists.newArrayList();
+    final List<Map.Entry<String, RelDataType>> fieldList = new ArrayList<>();
 
     for (int i = 0; i < selectItems.size(); i++) {
       SqlNode selectItem = selectItems.get(i);
@@ -4127,7 +4187,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
           return null;
         }
         final List<String> origin =
-            Lists.newArrayList(table.getQualifiedName());
+            new ArrayList<>(table.getQualifiedName());
         for (String name : qualified.suffix()) {
           namespace = namespace.lookupChild(name);
           if (namespace == null) {
@@ -4408,8 +4468,26 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
         return call.accept(this);
       }
       final SqlIdentifier fqId = getScope().fullyQualify(id).identifier;
-      validator.setOriginal(fqId, id);
-      return fqId;
+      SqlNode expandedExpr = fqId;
+      // Convert a column ref into ITEM(*, 'col_name').
+      // select col_name from (select * from dynTable)
+      // SqlIdentifier "col_name" would be resolved to a dynamic star field in dynTable's rowType.
+      // Expand such SqlIdentifier to ITEM operator.
+      if (DynamicRecordType.isDynamicStarColName(Util.last(fqId.names))
+        && !DynamicRecordType.isDynamicStarColName(Util.last(id.names))) {
+        SqlNode[] inputs = new SqlNode[2];
+        inputs[0] = fqId;
+        inputs[1] = SqlLiteral.createCharString(
+          Util.last(id.names),
+          id.getParserPosition());
+        SqlBasicCall item_call = new SqlBasicCall(
+          SqlStdOperatorTable.ITEM,
+          inputs,
+          id.getParserPosition());
+        expandedExpr = item_call;
+      }
+      validator.setOriginal(expandedExpr, id);
+      return expandedExpr;
     }
 
     @Override protected SqlNode visitScoped(SqlCall call) {
@@ -4417,6 +4495,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       case SCALAR_QUERY:
       case CURRENT_VALUE:
       case NEXT_VALUE:
+      case WITH:
         return call;
       }
       // Only visits arguments which are expressions. We don't want to

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/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 922512b..ec67bc9 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
@@ -660,6 +660,13 @@ public class SqlValidatorUtil {
     }
 
     public SqlNode visit(SqlIdentifier id) {
+      // First check for builtin functions which don't have parentheses,
+      // like "LOCALTIME".
+      final SqlCall call = SqlUtil.makeCall(getScope().getValidator().getOperatorTable(), id);
+      if (call != null) {
+        return call;
+      }
+
       return getScope().fullyQualify(id).identifier;
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java b/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
index b0a0be7..7ce1ac4 100644
--- a/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
+++ b/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
@@ -271,7 +271,7 @@ public class CalciteRemoteDriverTest {
   @Test public void testRemoteTypeInfo() throws Exception {
     CalciteAssert.hr().with(REMOTE_CONNECTION_FACTORY)
         .metaData(GET_TYPEINFO)
-        .returns(CalciteAssert.checkResultCount(30));
+        .returns(CalciteAssert.checkResultCount(31));
   }
 
   @Test public void testRemoteTableTypes() throws Exception {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/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 502e64d..c58f19f 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -32,11 +32,13 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.logical.LogicalFilter;
 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.RelDataTypeComparability;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.rel.type.RelRecordType;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
@@ -78,6 +80,7 @@ import com.google.common.collect.Sets;
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.util.AbstractList;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -571,6 +574,10 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     public String getCatalogName() {
       return DEFAULT_CATALOG;
     }
+
+    public String getName() {
+      return name;
+    }
   }
 
   /**
@@ -581,9 +588,9 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     protected final MockCatalogReader catalogReader;
     private final boolean stream;
     private final double rowCount;
-    private final List<Map.Entry<String, RelDataType>> columnList =
-        Lists.newArrayList();
-    private RelDataType rowType;
+    protected final List<Map.Entry<String, RelDataType>> columnList =
+        new ArrayList<>();
+    protected RelDataType rowType;
     private List<RelCollation> collationList;
     protected final List<String> names;
     private final Set<String> monotonicColumnSet = Sets.newHashSet();
@@ -713,6 +720,34 @@ public class MockCatalogReader implements Prepare.CatalogReader {
       return table;
     }
   }
+
+  /**
+   * Mock implementation of
+   * {@link org.apache.calcite.prepare.Prepare.PreparingTable} with dynamic record type.
+   */
+  public static class MockDynamicTable extends MockTable {
+    public 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);
+    }
+
+    /**
+     * Recreates an immutable rowType, if the table has Dynamic Record Type,
+     * when converts table to Rel.
+     */
+    public RelNode toRel(ToRelContext context) {
+      if (rowType.isDynamicStruct()) {
+        rowType = new RelRecordType(rowType.getFieldList());
+      }
+      return super.toRel(context);
+    }
+  }
+
+
 }
 
 // End MockCatalogReader.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/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 7a6efa6..122bf3d 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -16,15 +16,21 @@
  */
 package org.apache.calcite.test;
 
+import org.apache.calcite.prepare.Prepare;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelVisitor;
 import org.apache.calcite.rel.externalize.RelXmlWriter;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Function;
+
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -1731,6 +1737,167 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   }
 
   /**
+   * Test case for Dynamic Table / Dynamic Star support
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
+   */
+  @Test
+  public void testSelectFromDynamicTable() throws Exception {
+    Tester myTester = getTesterWithDynamicTable();
+    final String sql = "select n_nationkey, n_name from SALES.NATION";
+    myTester.assertConvertsTo(sql, "${plan}");
+  }
+
+  /**
+   * Test case for Dynamic Table / Dynamic Star support
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
+   */
+  @Test
+  public void testSelectStarFromDynamicTable() throws Exception {
+    Tester myTester = getTesterWithDynamicTable();
+    final String sql = "select * from SALES.NATION";
+    myTester.assertConvertsTo(sql, "${plan}");
+  }
+
+  /**
+   * Test case for Dynamic Table / Dynamic Star support
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
+   */
+  @Test
+  public void testReferDynamicStarInSelectOB() throws Exception {
+    Tester myTester = getTesterWithDynamicTable();
+    final String sql = "select n_nationkey, n_name from (select * from SALES.NATION) \n"
+        + " order by n_regionkey";
+    myTester.assertConvertsTo(sql, "${plan}");
+  }
+
+  /**
+   * Test case for Dynamic Table / Dynamic Star support
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
+   */
+  @Test
+  public void testDynamicStarInTableJoin() throws Exception {
+    Tester myTester = getTesterWithDynamicTable();
+    final String sql = "select * from "
+        + " (select * from SALES.NATION) T1, "
+        + " (SELECT * from SALES.CUSTOMER) T2 "
+        + " where T1.n_nationkey = T2.c_nationkey";
+    myTester.assertConvertsTo(sql, "${plan}");
+  }
+
+  /**
+   * Test case for Dynamic Table / Dynamic Star support
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
+   */
+  @Test
+  public void testReferDynamicStarInSelectWhereGB() throws Exception {
+    Tester myTester = getTesterWithDynamicTable();
+    final String sql = "select n_regionkey, count(*) as cnt from "
+        + "(select * from SALES.NATION) where n_nationkey > 5 "
+        + "group by n_regionkey";
+    myTester.assertConvertsTo(sql, "${plan}");
+  }
+
+  /**
+   * Test case for Dynamic Table / Dynamic Star support
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
+   */
+  @Test
+  public void testDynamicStarInJoinAndSubQ() throws Exception {
+    Tester myTester = getTesterWithDynamicTable();
+    final String sql = "select * from "
+        + " (select * from SALES.NATION T1, "
+        + " SALES.CUSTOMER T2 where T1.n_nationkey = T2.c_nationkey)";
+    myTester.assertConvertsTo(sql, "${plan}");
+  }
+
+  /**
+   * Test case for Dynamic Table / Dynamic Star support
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
+   */
+  @Test
+  public void testStarJoinStaticDynTable() throws Exception {
+    Tester myTester = getTesterWithDynamicTable();
+    final String sql = "select * from SALES.NATION N, SALES.REGION as R "
+        + "where N.n_regionkey = R.r_regionkey";
+    myTester.assertConvertsTo(sql, "${plan}");
+  }
+
+  /**
+   * Test case for Dynamic Table / Dynamic Star support
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
+   */
+  @Test
+  public void testGrpByColFromStarInSubQuery() throws Exception {
+    Tester myTester = getTesterWithDynamicTable();
+    final String sql = "SELECT n.n_nationkey AS col "
+        + " from (SELECT * FROM SALES.NATION) as n "
+        + " group by n.n_nationkey";
+    myTester.assertConvertsTo(sql, "${plan}");
+  }
+
+  /**
+   * Test case for Dynamic Table / Dynamic Star support
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
+   */
+  @Test
+  public void testDynStarInExistSubQ() throws Exception {
+    Tester myTester = getTesterWithDynamicTable();
+    final String sql = "select * from SALES.REGION where exists (select * from SALES.NATION)";
+    myTester.assertConvertsTo(sql, "${plan}");
+  }
+
+  /**
+   * Test case for Dynamic Table / Dynamic Star support
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
+   */
+  @Test
+  public void testSelStarOrderBy() throws Exception {
+    Tester myTester = getTesterWithDynamicTable();
+    final String sql = "SELECT * from SALES.NATION order by n_nationkey";
+    myTester.assertConvertsTo(sql, "${plan}");
+  }
+
+  private Tester getTesterWithDynamicTable() {
+    return tester.withCatalogReaderFactory(
+        new Function<RelDataTypeFactory, Prepare.CatalogReader>() {
+          public Prepare.CatalogReader apply(RelDataTypeFactory typeFactory) {
+            return new MockCatalogReader(typeFactory, true) {
+              @Override public MockCatalogReader init() {
+                // CREATE SCHEMA "SALES;
+                // CREATE DYNAMIC TABLE "NATION"
+                // CREATE DYNAMIC TABLE "CUSTOMER"
+
+                MockSchema schema = new MockSchema("SALES");
+                registerSchema(schema);
+
+                MockTable nationTable = new MockDynamicTable(this, schema.getCatalogName(),
+                    schema.getName(), "NATION", false, 100);
+                registerTable(nationTable);
+
+                MockTable customerTable = new MockDynamicTable(this, schema.getCatalogName(),
+                    schema.getName(), "CUSTOMER", false, 100);
+                registerTable(customerTable);
+
+                // CREATE TABLE "REGION" - static table with known schema.
+                final RelDataType intType =
+                    typeFactory.createSqlType(SqlTypeName.INTEGER);
+                final RelDataType varcharType =
+                    typeFactory.createSqlType(SqlTypeName.VARCHAR);
+
+                MockTable regionTable = MockTable.create(this, schema, "REGION", false, 100);
+                regionTable.addColumn("R_REGIONKEY", intType);
+                regionTable.addColumn("R_NAME", varcharType);
+                regionTable.addColumn("R_COMMENT", varcharType);
+                registerTable(regionTable);
+                return this;
+              }
+              // CHECKSTYLE: IGNORE 1
+            }.init();
+          }
+        });
+  }
+
+  /**
    * Visitor that checks that every {@link RelNode} in a tree is valid.
    *
    * @see RelNode#isValid(org.apache.calcite.util.Litmus)

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
index 3d7f609..40c6d47 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -575,7 +575,7 @@ public abstract class SqlToRelTestBase {
         RelDataTypeFactory typeFactory) {
       return new FarragoTestValidator(
           getOperatorTable(),
-          createCatalogReader(typeFactory),
+          catalogReader,
           typeFactory,
           getConformance());
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/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 d81a89f..bc8c5e4 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -3954,7 +3954,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     winSql("select *\n"
         + " from emp\n"
         + " join dept on emp.deptno = dept.deptno\n"
-        + " and ^sum(sal) over (partition by deptno\n"
+        + " and ^sum(sal) over (partition by emp.deptno\n"
         + "    order by empno\n"
         + "    rows 3 preceding)^ = dept.deptno + 40\n"
         + "order by deptno")
@@ -5932,7 +5932,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // Group by
     checkFails(
         "select 1 from emp group by deptno order by ^empno^",
-        "Expression 'EMP\\.EMPNO' is not being grouped");
+        "Expression 'EMPNO' is not being grouped");
 
     // order by can contain aggregate expressions
     check("select empno from emp "
@@ -5943,7 +5943,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     checkFails(
         "select sum(sal) from emp having count(*) > 3 order by ^empno^",
-        "Expression 'EMP\\.EMPNO' is not being grouped");
+        "Expression 'EMPNO' is not being grouped");
 
     check("select sum(sal) from emp having count(*) > 3 order by sum(deptno)");
 
@@ -5951,11 +5951,11 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     checkFails(
         "select distinct deptno from emp group by deptno order by ^empno^",
-        "Expression 'EMP\\.EMPNO' is not in the select clause");
+        "Expression 'EMPNO' is not in the select clause");
 
     checkFails(
         "select distinct deptno from emp group by deptno order by deptno, ^empno^",
-        "Expression 'EMP\\.EMPNO' is not in the select clause");
+        "Expression 'EMPNO' is not in the select clause");
 
     check("select distinct deptno from emp group by deptno order by deptno");
 
@@ -5987,7 +5987,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkFails(
         "select distinct cast(empno as bigint) "
             + "from emp order by ^empno^",
-        "Expression 'EMP\\.EMPNO' is not in the select clause");
+        "Expression 'EMPNO' is not in the select clause");
     checkFails(
         "select distinct cast(empno as bigint) "
             + "from emp order by ^emp.empno^",
@@ -6073,10 +6073,10 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "Aggregate expression is illegal in ORDER BY clause of non-aggregating SELECT");
     checkFails("SELECT DISTINCT deptno from emp\n"
         + "GROUP BY deptno ORDER BY deptno, ^sum(empno)^",
-        "Expression 'SUM\\(`EMP`\\.`EMPNO`\\)' is not in the select clause");
+        "Expression 'SUM\\(`EMPNO`\\)' is not in the select clause");
     checkFails("SELECT DISTINCT deptno, min(empno) from emp\n"
         + "GROUP BY deptno ORDER BY deptno, ^sum(empno)^",
-        "Expression 'SUM\\(`EMP`\\.`EMPNO`\\)' is not in the select clause");
+        "Expression 'SUM\\(`EMPNO`\\)' is not in the select clause");
     check("SELECT DISTINCT deptno, sum(empno) from emp\n"
         + "GROUP BY deptno ORDER BY deptno, sum(empno)");
   }
@@ -6822,17 +6822,17 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // similar validation for SELECT DISTINCT and GROUP BY
     checkFails(
         "SELECT deptno FROM emp GROUP BY deptno ORDER BY deptno, ^empno^",
-        "Expression 'EMP\\.EMPNO' is not being grouped");
+        "Expression 'EMPNO' is not being grouped");
     checkFails(
         "SELECT DISTINCT deptno from emp ORDER BY deptno, ^empno^",
-        "Expression 'EMP\\.EMPNO' is not in the select clause");
+        "Expression 'EMPNO' is not in the select clause");
     check("SELECT DISTINCT deptno from emp ORDER BY deptno + 2");
 
     // The ORDER BY clause works on what is projected by DISTINCT - even if
     // GROUP BY is present.
     checkFails(
         "SELECT DISTINCT deptno FROM emp GROUP BY deptno, empno ORDER BY deptno, ^empno^",
-        "Expression 'EMP\\.EMPNO' is not in the select clause");
+        "Expression 'EMPNO' is not in the select clause");
 
     // redundant distinct; same query is in unitsql/optimizer/distinct.sql
     check("select distinct * from (\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/9bd7d755/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 0f9c715..1d49809 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -3273,4 +3273,142 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testSelectFromDynamicTable">
+        <Resource name="sql">
+            <![CDATA[select n_nationkey, n_name from SALES.NATION]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(N_NATIONKEY=[$0], N_NAME=[$1])
+  LogicalTableScan(table=[[CATALOG, SALES, NATION]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testSelectStarFromDynamicTable">
+        <Resource name="sql">
+            <![CDATA[select * from SALES.NATION]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(**=[$0])
+  LogicalTableScan(table=[[CATALOG, SALES, NATION]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testReferDynamicStarInSelectOB">
+        <Resource name="sql">
+            <![CDATA[select n_nationkey, n_name from (select * from SALES.NATION) order by n_regionkey]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(N_NATIONKEY=[$0], N_NAME=[$1])
+  LogicalSort(sort0=[$2], dir0=[ASC])
+    LogicalProject(N_NATIONKEY=[ITEM($0, 'N_NATIONKEY')], N_NAME=[ITEM($0, 'N_NAME')], EXPR$2=[ITEM($0, 'N_REGIONKEY')])
+      LogicalProject(**=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, NATION]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testDynamicStarInTableJoin">
+        <Resource name="sql">
+            <![CDATA[select * from (select * from SALES.NATION) T1, (SELECT * from SALES.CUSTOMER) T2 where T1.n_nationkey = T2.c_nationkey ]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(**=[$0], **0=[$1])
+  LogicalFilter(condition=[=(ITEM($0, 'N_NATIONKEY'), ITEM($1, 'C_NATIONKEY'))])
+    LogicalJoin(condition=[true], joinType=[inner])
+      LogicalProject(**=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, NATION]])
+      LogicalProject(**=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, CUSTOMER]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testReferDynamicStarInSelectWhereGB">
+        <Resource name="sql">
+            <![CDATA[select n_regionkey, count(*) as cnt from (select * from SALES.NATION) where n_nationkey > 5 group by n_regionkey ]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalAggregate(group=[{0}], CNT=[COUNT()])
+  LogicalProject(N_REGIONKEY=[ITEM($0, 'N_REGIONKEY')])
+    LogicalFilter(condition=[>(CAST(ITEM($0, 'N_NATIONKEY')):INTEGER, 5)])
+      LogicalProject(**=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, NATION]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testDynamicStarInJoinAndSubQ">
+        <Resource name="sql">
+            <![CDATA[select * from (select * from SALES.NATION T1, SALES.CUSTOMER T2 where T1.n_nationkey = T2.c_nationkey) ]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(**=[$0], **0=[$1])
+  LogicalProject(**=[$1], **0=[$3])
+    LogicalFilter(condition=[=($0, $2)])
+      LogicalJoin(condition=[true], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, NATION]])
+        LogicalTableScan(table=[[CATALOG, SALES, CUSTOMER]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testStarJoinStaticDynTable">
+        <Resource name="sql">
+            <![CDATA[select * from SALES.NATION N, SALES.REGION R where N.n_regionkey = R.r_regionkey ]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(**=[$1], R_REGIONKEY=[$2], R_NAME=[$3], R_COMMENT=[$4])
+  LogicalFilter(condition=[=(CAST($0):INTEGER, $2)])
+    LogicalJoin(condition=[true], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, SALES, NATION]])
+      LogicalTableScan(table=[[CATALOG, SALES, REGION]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testGrpByColFromStarInSubQuery">
+        <Resource name="sql">
+            <![CDATA[select * from SALES.NATION N, SALES.REGION R where N.n_regionkey = R.r_regionkey ]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalAggregate(group=[{0}])
+  LogicalProject(COL=[ITEM($0, 'N_NATIONKEY')])
+    LogicalProject(**=[$0])
+      LogicalTableScan(table=[[CATALOG, SALES, NATION]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testDynStarInExistSubQ">
+        <Resource name="sql">
+            <![CDATA[select * from SALES.REGION where exists (select * from SALES.NATION)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(R_REGIONKEY=[$0], R_NAME=[$1], R_COMMENT=[$2])
+  LogicalFilter(condition=[IS NOT NULL($3)])
+    LogicalJoin(condition=[true], joinType=[left])
+      LogicalTableScan(table=[[CATALOG, SALES, REGION]])
+      LogicalAggregate(group=[{}], agg#0=[MIN($0)])
+        LogicalProject($f0=[true])
+          LogicalProject(**=[$0])
+            LogicalTableScan(table=[[CATALOG, SALES, NATION]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testSelStarOrderBy">
+        <Resource name="sql">
+            <![CDATA[SELECT * from SALES.NATION order by n_nationkey]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(**=[$0])
+  LogicalSort(sort0=[$1], dir0=[ASC])
+    LogicalProject(**=[$0], N_NATIONKEY=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, NATION]])
+]]>
+        </Resource>
+    </TestCase>
 </Root>