You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by ma...@apache.org on 2016/12/01 08:27:58 UTC

calcite git commit: [CALCITE-1425] Support two-level column structure in INSERT/UPDATE/MERGE

Repository: calcite
Updated Branches:
  refs/heads/master 1cea152e2 -> 46654ad2a


[CALCITE-1425] Support two-level column structure in INSERT/UPDATE/MERGE


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

Branch: refs/heads/master
Commit: 46654ad2a6db9f3b1d03eaa5d53645ee5ed591a1
Parents: 1cea152
Author: maryannxue <ma...@gmail.com>
Authored: Thu Dec 1 00:27:47 2016 -0800
Committer: maryannxue <ma...@gmail.com>
Committed: Thu Dec 1 00:27:47 2016 -0800

----------------------------------------------------------------------
 .../schema/CustomColumnResolvingTable.java      |  58 +++
 .../calcite/schema/CustomExpansionTable.java    |  42 --
 .../org/apache/calcite/sql/SqlIdentifier.java   |  10 +
 .../java/org/apache/calcite/sql/SqlInsert.java  |   4 -
 .../calcite/sql/advise/SqlAdvisorValidator.java |   7 -
 .../calcite/sql/validate/DelegatingScope.java   |  61 ++-
 .../sql/validate/IdentifierNamespace.java       |   2 +-
 .../apache/calcite/sql/validate/ListScope.java  |  19 -
 .../calcite/sql/validate/SqlValidator.java      |   7 -
 .../calcite/sql/validate/SqlValidatorImpl.java  | 137 +++---
 .../calcite/sql/validate/SqlValidatorUtil.java  |  34 ++
 .../calcite/sql2rel/SqlToRelConverter.java      |  22 +-
 .../apache/calcite/test/MockCatalogReader.java  | 449 ++++++++++++-------
 .../apache/calcite/test/RelOptRulesTest.java    |   6 +-
 .../calcite/test/SqlToRelConverterTest.java     |  41 +-
 .../apache/calcite/test/SqlValidatorTest.java   |  51 ++-
 .../org/apache/calcite/test/RelOptRulesTest.xml |  94 ++--
 .../calcite/test/SqlToRelConverterTest.xml      |  94 ++--
 18 files changed, 673 insertions(+), 465 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/core/src/main/java/org/apache/calcite/schema/CustomColumnResolvingTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/CustomColumnResolvingTable.java b/core/src/main/java/org/apache/calcite/schema/CustomColumnResolvingTable.java
new file mode 100644
index 0000000..322a297
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/schema/CustomColumnResolvingTable.java
@@ -0,0 +1,58 @@
+/*
+ * 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.schema;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.util.Pair;
+
+import java.util.List;
+
+/**
+ * Extension to {@link Table} that specifies a custom way to resolve column
+ * names.
+ *
+ * <p>It is optional for a Table to implement this interface. If Table does
+ * not implement this interface, column resolving will be performed in the
+ * default way.</p>
+ *
+ * <p><strong>NOTE: This class is experimental and subject to
+ * change/removal without notice</strong>.</p>
+ */
+public interface CustomColumnResolvingTable extends Table {
+
+  /**
+   * Resolve a column based on the name components. One or more the input name
+   * components can be resolved to one field in the table row type, along with
+   * a remainder list of name components which have not been resolved within
+   * this call, and which in turn can be potentially resolved as sub-field
+   * names. In the meantime, this method can return multiple matches, which is
+   * a list of pairs containing the resolved field and the remaining name
+   * components.
+   *
+   * @param rowType     the table row type
+   * @param typeFactory the type factory
+   * @param names       the name components to be resolved
+   * @return  a list of pairs containing the resolved field and the remaining
+   *          name components.
+   */
+  List<Pair<RelDataTypeField, List<String>>> resolveColumn(
+      RelDataType rowType, RelDataTypeFactory typeFactory, List<String> names);
+}
+
+// End CustomColumnResolvingTable.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/core/src/main/java/org/apache/calcite/schema/CustomExpansionTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/CustomExpansionTable.java b/core/src/main/java/org/apache/calcite/schema/CustomExpansionTable.java
deleted file mode 100644
index 21ac696..0000000
--- a/core/src/main/java/org/apache/calcite/schema/CustomExpansionTable.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.schema;
-
-import java.util.List;
-
-/**
- * Extension to {@link Table} that specifies a list of column names for
- * custom star expansion. The columns specified in the list can be any
- * top-level column from the Table or any field or nested field under a
- * top-level column, thus each column name is returned as a list of String
- * objects representing the full name of the column or field. This expansion
- * list will also be used as target columns in INSERT if the original target
- * column list is not present.
- *
- * <p>It is optional for a Table to implement this interface. If Table does
- * not implement this interface, star expansion will be performed in the
- * default way according to the Table's row type.</p>
- *
- * <p><strong>NOTE: This class is experimental and subject to
- * change/removal without notice</strong>.</p>
- */
-public interface CustomExpansionTable extends Table {
-  /** Returns a list of column names for custom star expansion. */
-  List<List<String>> getCustomStarExpansion();
-}
-
-// End CustomExpansionTable.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/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 d070150..7e2294c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
@@ -258,6 +258,16 @@ public class SqlIdentifier extends SqlNode {
     return new SqlIdentifier(names, collation, pos2, componentPositions);
   }
 
+  /**
+   * Creates an identifier that consists of this identifier plus a wildcard star.
+   * Does not modify this identifier.
+   */
+  public SqlIdentifier plusStar() {
+    final SqlIdentifier id = this.plus("*", SqlParserPos.ZERO);
+    return new SqlIdentifier(Lists.transform(id.names, STAR_TO_EMPTY), null, id.pos,
+        id.componentPositions);
+  }
+
   /** Creates an identifier that consists of all but the last {@code n}
    * name segments of this one. */
   public SqlIdentifier skipLast(int n) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlInsert.java b/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
index 1c9030e..19472a7 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
@@ -120,10 +120,6 @@ public class SqlInsert extends SqlCall {
     return columnList;
   }
 
-  public void setTargetColumnList(SqlNodeList columnList) {
-    this.columnList = columnList;
-  }
-
   public final SqlNode getModifierNode(SqlInsertKeyword modifier) {
     for (SqlNode keyword : keywords) {
       SqlInsertKeyword keyword2 =

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorValidator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorValidator.java b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorValidator.java
index 90f68cc..4d07e79 100644
--- a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorValidator.java
+++ b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorValidator.java
@@ -201,13 +201,6 @@ public class SqlAdvisorValidator extends SqlValidatorImpl {
     return true;
   }
 
-  public boolean shouldUseCustomStarExpansion() {
-    // Disable custom star expansion otherwise SqlValidatorNamespace.getTable()
-    // could be called on a SqlValidatorNamespace that was not successfully
-    // validated.
-    return false;
-  }
-
   protected boolean shouldAllowOverRelation() {
     return true; // no reason not to be lenient
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/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 a1f63b6..d9fd167 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,10 +16,13 @@
  */
 package org.apache.calcite.sql.validate;
 
+import org.apache.calcite.prepare.Prepare;
 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.rel.type.StructKind;
+import org.apache.calcite.schema.CustomColumnResolvingTable;
+import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
@@ -93,6 +96,26 @@ public abstract class DelegatingScope implements SqlValidatorScope {
     }
     final RelDataType rowType = ns.getRowType();
     if (rowType.isStruct()) {
+      SqlValidatorTable validatorTable = ns.getTable();
+      if (validatorTable instanceof Prepare.PreparingTable) {
+        Table t = ((Prepare.PreparingTable) validatorTable).unwrap(Table.class);
+        if (t instanceof CustomColumnResolvingTable) {
+          final List<Pair<RelDataTypeField, List<String>>> entries =
+              ((CustomColumnResolvingTable) t).resolveColumn(
+                  rowType, validator.getTypeFactory(), names);
+          for (Pair<RelDataTypeField, List<String>> entry : entries) {
+            final RelDataTypeField field = entry.getKey();
+            final List<String> remainder = entry.getValue();
+            final SqlValidatorNamespace ns2 =
+                new FieldNamespace(validator, field.getType());
+            final Step path2 = path.add(rowType, field.getIndex(),
+                StructKind.FULLY_QUALIFIED);
+            resolveInNamespace(ns2, remainder, path2, resolved);
+          }
+          return;
+        }
+      }
+
       final String name = names.get(0);
       final RelDataTypeField field0 =
           validator.catalogReader.field(rowType, name);
@@ -199,6 +222,9 @@ public abstract class DelegatingScope implements SqlValidatorScope {
       final String tableName = pair.left;
       final SqlValidatorNamespace namespace = pair.right;
 
+      final ResolvedImpl resolved = new ResolvedImpl();
+      resolveInNamespace(namespace,
+          identifier.names, resolved.emptyPath(), resolved);
       final RelDataTypeField field =
           validator.catalogReader.field(namespace.getRowType(), columnName);
       if (field != null) {
@@ -273,20 +299,19 @@ public abstract class DelegatingScope implements SqlValidatorScope {
                 ++i;
                 ++size;
               }
+              break;
+            default:
+              // Throw an error if the table was not found.
+              // If one or more of the child namespaces allows peeking
+              // (e.g. if they are Phoenix column families) then we relax the SQL
+              // standard requirement that record fields are qualified by table alias.
+              final SqlIdentifier prefix = identifier.skipLast(1);
+              throw validator.newValidationError(prefix,
+                  RESOURCE.tableNameNotFound(prefix.toString()));
             }
           }
         }
         }
-
-        // Throw an error if the table was not found.
-        // If one or more of the child namespaces allows peeking
-        // (e.g. if they are Phoenix column families) then we relax the SQL
-        // standard requirement that record fields are qualified by table alias.
-        if (!hasLiberalChild()) {
-          final SqlIdentifier prefix1 = identifier.skipLast(1);
-          throw validator.newValidationError(prefix1,
-              RESOURCE.tableNameNotFound(prefix1.toString()));
-        }
       }
 
       // If a table alias is part of the identifier, make sure that the table
@@ -370,6 +395,11 @@ public abstract class DelegatingScope implements SqlValidatorScope {
       // and check that references to dynamic stars ("**") are unambiguous.
       int k = i;
       for (Step step : path.steps()) {
+        final String name = identifier.names.get(k);
+        if (step.i < 0) {
+          throw validator.newValidationError(
+              identifier, RESOURCE.columnNotFound(name));
+        }
         final RelDataTypeField field0 =
             step.rowType.getFieldList().get(step.i);
         final String fieldName = field0.getName();
@@ -379,7 +409,6 @@ public abstract class DelegatingScope implements SqlValidatorScope {
           identifier = identifier.add(k, fieldName, SqlParserPos.ZERO);
           break;
         default:
-          final String name = identifier.names.get(k);
           if (!fieldName.equals(name)) {
             identifier = identifier.setName(k, fieldName);
           }
@@ -391,6 +420,12 @@ public abstract class DelegatingScope implements SqlValidatorScope {
         ++k;
       }
 
+      // Multiple name components may have been resolved as one step by
+      // CustomResolvingTable.
+      if (identifier.names.size() > k) {
+        identifier = identifier.getComponent(0, k);
+      }
+
       if (i > 1) {
         // Simplify overqualified identifiers.
         // For example, schema.emp.deptno becomes emp.deptno.
@@ -410,10 +445,6 @@ public abstract class DelegatingScope implements SqlValidatorScope {
     }
   }
 
-  protected boolean hasLiberalChild() {
-    return false;
-  }
-
   public void validateExpr(SqlNode expr) {
     // Do not delegate to parent. An expression valid in this scope may not
     // be valid in the parent scope.

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
index 5133f50..5282ddb 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
@@ -192,7 +192,7 @@ public class IdentifierNamespace extends AbstractNamespace {
   }
 
   @Override public SqlValidatorTable getTable() {
-    return resolve().getTable();
+    return resolvedNamespace == null ? null : resolve().getTable();
   }
 
   public List<Pair<SqlNode, SqlMonotonicity>> getMonotonicExprs() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java b/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java
index c4d292f..444da8d 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java
@@ -141,25 +141,6 @@ public abstract class ListScope extends DelegatingScope {
     return map;
   }
 
-  @Override protected boolean hasLiberalChild() {
-    for (Pair<String, SqlValidatorNamespace> child : children) {
-      final RelDataType rowType = child.right.getRowType();
-      switch (rowType.getStructKind()) {
-      case PEEK_FIELDS:
-      case PEEK_FIELDS_DEFAULT:
-        return true;
-      }
-      for (RelDataTypeField field : rowType.getFieldList()) {
-        switch (field.getType().getStructKind()) {
-        case PEEK_FIELDS:
-        case PEEK_FIELDS_DEFAULT:
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
   @Override public void resolve(List<String> names, boolean deep,
       Resolved resolved) {
     // First resolve by looking through the child namespaces.

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
index cc14c36..215b96c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
@@ -617,13 +617,6 @@ public interface SqlValidator {
   boolean shouldExpandIdentifiers();
 
   /**
-   * Returns whether to use a Table's custom star expansion.
-   *
-   * @return true if custom star expansion should be used; false otherwise.
-   */
-  boolean shouldUseCustomStarExpansion();
-
-  /**
    * Enables or disables rewrite of "macro-like" calls such as COALESCE.
    *
    * @param rewriteCalls new setting

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/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 b2dada1..c2c24c3 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,7 +19,6 @@ 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.prepare.Prepare;
 import org.apache.calcite.rel.type.DynamicRecordType;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -30,7 +29,6 @@ import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.Feature;
 import org.apache.calcite.runtime.Resources;
-import org.apache.calcite.schema.CustomExpansionTable;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.JoinConditionType;
 import org.apache.calcite.sql.JoinType;
@@ -462,42 +460,26 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
                scope,
                includeSystemVars);
         } else {
-          final List<List<String>> customStarExpansion =
-              getCustomStarExpansion(p.right);
-          if (customStarExpansion != null) {
-            for (List<String> names : customStarExpansion) {
-              SqlIdentifier exp = new SqlIdentifier(
-                  Lists.asList(p.left, names.toArray(new String[names.size()])),
-                  startPosition);
-              addToSelectList(
-                  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);
-            }
+          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 SqlIdentifier exp =
+                new SqlIdentifier(
+                    ImmutableList.of(p.left, columnName),
+                    startPosition);
+            addOrExpandField(
+                selectItems,
+                aliases,
+                types,
+                includeSystemVars,
+                scope,
+                exp,
+                field);
           }
         }
       }
@@ -530,13 +512,14 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
           String columnName = field.getName();
 
           // TODO: do real implicit collation here
-          addToSelectList(
+          addOrExpandField(
               selectItems,
               aliases,
               types,
-              prefixId.plus(columnName, startPosition),
+              includeSystemVars,
               scope,
-              includeSystemVars);
+              prefixId.plus(columnName, startPosition),
+              field);
         }
       } else {
         throw newValidationError(prefixId, RESOURCE.starRequiresRecordType());
@@ -545,18 +528,33 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     }
   }
 
-  private List<List<String>> getCustomStarExpansion(SqlValidatorNamespace ns) {
-    if (!shouldUseCustomStarExpansion()) {
-      return null;
-    }
-    final SqlValidatorTable table = ns.getTable();
-    if (table instanceof Prepare.PreparingTable) {
-      Table t = ((Prepare.PreparingTable) table).unwrap(Table.class);
-      if (t instanceof CustomExpansionTable) {
-        return ((CustomExpansionTable) t).getCustomStarExpansion();
-      }
+  private boolean addOrExpandField(List<SqlNode> selectItems, Set<String> aliases,
+      List<Map.Entry<String, RelDataType>> types, boolean includeSystemVars,
+      SelectScope scope, SqlIdentifier id, RelDataTypeField field) {
+    switch (field.getType().getStructKind()) {
+    case PEEK_FIELDS:
+    case PEEK_FIELDS_DEFAULT:
+      final SqlNode starExp = id.plusStar();
+      expandStar(
+          selectItems,
+          aliases,
+          types,
+          includeSystemVars,
+          scope,
+          starExp);
+      return true;
+
+    default:
+      addToSelectList(
+          selectItems,
+          aliases,
+          types,
+          id,
+          scope,
+          includeSystemVars);
     }
-    return null;
+
+    return false;
   }
 
   public SqlNode validate(SqlNode topNode) {
@@ -1761,10 +1759,6 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     return expandIdentifiers;
   }
 
-  public boolean shouldUseCustomStarExpansion() {
-    return true;
-  }
-
   protected boolean shouldAllowIntermediateOrderBy() {
     return true;
   }
@@ -3719,12 +3713,16 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       }
     }
     final Set<Integer> assignedFields = new HashSet<>();
+    final RelOptTable relOptTable = table instanceof RelOptTable
+        ? ((RelOptTable) table) : null;
     for (SqlNode node : targetColumnList) {
       SqlIdentifier id = (SqlIdentifier) node;
-      String name = id.getSimple();
-      RelDataTypeField targetField = catalogReader.field(baseRowType, name);
+      RelDataTypeField targetField =
+          SqlValidatorUtil.getTargetField(
+              baseRowType, typeFactory, id, catalogReader, relOptTable);
       if (targetField == null) {
-        throw newValidationError(id, RESOURCE.unknownTargetColumn(name));
+        throw newValidationError(id,
+            RESOURCE.unknownTargetColumn(id.toString()));
       }
       if (!assignedFields.add(targetField.getIndex())) {
         throw newValidationError(id,
@@ -3740,11 +3738,6 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     validateNamespace(targetNamespace, unknownType);
     SqlValidatorTable table = targetNamespace.getTable();
 
-    // If the INSERT does not have a target column list and the target
-    // table specifies a custom star expansion list, we will set the new
-    // target column list as the star expansion list.
-    rewriteTargetColumnList(insert, targetNamespace);
-
     // INSERT has an optional column name list.  If present then
     // reduce the rowtype to the columns specified.  If not present
     // then the entire target rowtype is used.
@@ -3782,22 +3775,6 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     validateAccess(insert.getTargetTable(), table, SqlAccessEnum.INSERT);
   }
 
-  private void rewriteTargetColumnList(
-      SqlInsert insert, SqlValidatorNamespace ns) {
-    final List<List<String>> customStarExpansion = getCustomStarExpansion(ns);
-    if (customStarExpansion == null) {
-      return;
-    }
-
-    final List<SqlNode> targetColumnList = new ArrayList<>();
-    final SqlParserPos startPosition = insert.getParserPosition();
-    for (List<String> names : customStarExpansion) {
-      targetColumnList.add(new SqlIdentifier(names, startPosition));
-    }
-    insert.setTargetColumnList(
-        new SqlNodeList(targetColumnList, startPosition));
-  }
-
   private void checkFieldCount(
       SqlNode node,
       RelDataType logicalSourceRowType,

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/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 62410b3..954a84b 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
@@ -25,6 +25,8 @@ import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.schema.CustomColumnResolvingTable;
+import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDynamicParam;
@@ -40,6 +42,7 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Litmus;
+import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -436,6 +439,37 @@ public class SqlValidatorUtil {
   }
 
   /**
+   * Resolve a target column name in the target table.
+   *
+   * @return the target field or null if the name cannot be resolved
+   * @param rowType the target row type
+   * @param id      the target column identifier
+   * @param table   the target table or null if it is not a RelOptTable instance
+   */
+  public static RelDataTypeField getTargetField(
+      RelDataType rowType, RelDataTypeFactory typeFactory,
+      SqlIdentifier id, SqlValidatorCatalogReader catalogReader,
+      RelOptTable table) {
+    final Table t = table == null ? null : table.unwrap(Table.class);
+    if (!(t instanceof CustomColumnResolvingTable)) {
+      return catalogReader.field(rowType, id.getSimple());
+    }
+
+    final List<Pair<RelDataTypeField, List<String>>> entries =
+        ((CustomColumnResolvingTable) t).resolveColumn(
+            rowType, typeFactory, id.names);
+    switch (entries.size()) {
+    case 1:
+      if (!entries.get(0).getValue().isEmpty()) {
+        return null;
+      }
+      return entries.get(0).getKey();
+    default:
+      return null;
+    }
+  }
+
+  /**
    * Resolves a multi-part identifier such as "SCHEMA.EMP.EMPNO" to a
    * namespace. The returned namespace, never null, may represent a
    * schema, table, column, etc.

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/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 0624645..45994a0 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -3120,7 +3120,11 @@ public class SqlToRelConverter {
     } else {
       for (int i = 0; i < targetColumnList.size(); i++) {
         SqlIdentifier id = (SqlIdentifier) targetColumnList.get(i);
-        targetColumnNames.add(id.getSimple());
+        RelDataTypeField field =
+            SqlValidatorUtil.getTargetField(
+                targetRowType, typeFactory, id, catalogReader, targetTable);
+        assert field != null : "column " + id.toString() + " not found";
+        targetColumnNames.add(field.getName());
       }
     }
 
@@ -3142,10 +3146,14 @@ public class SqlToRelConverter {
 
     // convert update column list from SqlIdentifier to String
     final List<String> targetColumnNameList = new ArrayList<>();
+    final RelDataType targetRowType = targetTable.getRowType();
     for (SqlNode node : call.getTargetColumnList()) {
       SqlIdentifier id = (SqlIdentifier) node;
-      String name = id.getSimple();
-      targetColumnNameList.add(name);
+      RelDataTypeField field =
+          SqlValidatorUtil.getTargetField(
+              targetRowType, typeFactory, id, catalogReader, targetTable);
+      assert field != null : "column " + id.toString() + " not found";
+      targetColumnNameList.add(field.getName());
     }
 
     RelNode sourceRel = convertSelect(call.getSourceSelect(), false);
@@ -3159,12 +3167,16 @@ public class SqlToRelConverter {
 
     // convert update column list from SqlIdentifier to String
     final List<String> targetColumnNameList = new ArrayList<>();
+    final RelDataType targetRowType = targetTable.getRowType();
     SqlUpdate updateCall = call.getUpdateCall();
     if (updateCall != null) {
       for (SqlNode targetColumn : updateCall.getTargetColumnList()) {
         SqlIdentifier id = (SqlIdentifier) targetColumn;
-        String name = id.getSimple();
-        targetColumnNameList.add(name);
+        RelDataTypeField field =
+            SqlValidatorUtil.getTargetField(
+                targetRowType, typeFactory, id, catalogReader, targetTable);
+        assert field != null : "column " + id.toString() + " not found";
+        targetColumnNameList.add(field.getName());
       }
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/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 3e5246a..9dc0692 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.test;
 
+import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.linq4j.Queryable;
 import org.apache.calcite.linq4j.tree.Expression;
@@ -48,8 +49,7 @@ import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.schema.CustomExpansionTable;
-import org.apache.calcite.schema.ModifiableView;
+import org.apache.calcite.schema.CustomColumnResolvingTable;
 import org.apache.calcite.schema.Path;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
@@ -93,8 +93,10 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 
 /**
@@ -357,8 +359,8 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     //   FROM EMP
     //   WHERE DEPTNO = 20 AND SAL > 1000
     MockTable emp20View = new MockViewTable(this, salesSchema.getCatalogName(),
-        salesSchema.name, "EMP_20", false, 600, empTable, null,
-        ImmutableIntList.of(0, 1, 2, 3, 4, 5, 6, 8)) {
+        salesSchema.name, "EMP_20", false, 600, empTable,
+        ImmutableIntList.of(0, 1, 2, 3, 4, 5, 6, 8), null) {
 
       @Override public RexNode getConstraint(RexBuilder rexBuilder,
           RelDataType tableRowType) {
@@ -393,89 +395,55 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
     MockSchema structTypeSchema = new MockSchema("STRUCT");
     registerSchema(structTypeSchema);
-    MockTable structTypeTable = new MockTable(this,
+    final List<CompoundNameColumn> columns = Arrays.asList(
+        new CompoundNameColumn("", "K0", varchar20Type),
+        new CompoundNameColumn("", "C1", varchar20Type),
+        new CompoundNameColumn("F1", "A0", intType),
+        new CompoundNameColumn("F2", "A0", booleanType),
+        new CompoundNameColumn("F0", "C0", intType),
+        new CompoundNameColumn("F1", "C0", intTypeNull),
+        new CompoundNameColumn("F0", "C1", intType),
+        new CompoundNameColumn("F1", "C2", intType),
+        new CompoundNameColumn("F2", "C3", intType));
+    final CompoundNameColumnResolver structTypeTableResolver =
+        new CompoundNameColumnResolver(columns, "F0");
+    final MockTable structTypeTable = new MockTable(this,
         structTypeSchema.getCatalogName(), structTypeSchema.name,
-        "T", false, 100,
-        ImmutableList.<List<String>>of(
-            ImmutableList.of("K0"),
-            ImmutableList.of("C1"),
-            ImmutableList.of("F1", "A0"),
-            ImmutableList.of("F2", "A0"),
-            ImmutableList.of("F0", "C0"),
-            ImmutableList.of("F1", "C0"),
-            ImmutableList.of("F0", "C1"),
-            ImmutableList.of("F1", "C2"),
-            ImmutableList.of("F2", "C3")));
-    structTypeTable.addColumn("K0", varchar20Type);
-    structTypeTable.addColumn("C1", varchar20Type);
-    final RelDataType f0Type = typeFactory.builder()
-        .add("C0", intType)
-        .add("C1", intType)
-        .kind(StructKind.PEEK_FIELDS_DEFAULT)
-        .build();
-    structTypeTable.addColumn("F0", f0Type);
-    final RelDataType f1Type = typeFactory.builder()
-        .add("C0", intTypeNull)
-        .add("C2", intType)
-        .add("A0", intType)
-        .kind(StructKind.PEEK_FIELDS)
-        .build();
-    structTypeTable.addColumn("F1", f1Type);
-    final RelDataType f2Type = typeFactory.builder()
-        .add("C3", intType)
-        .add("A0", booleanType)
-        .kind(StructKind.PEEK_FIELDS)
-        .build();
-    structTypeTable.addColumn("F2", f2Type);
+        "T", false, 100, structTypeTableResolver);
+    for (CompoundNameColumn column : columns) {
+      structTypeTable.addColumn(column.getName(), column.type);
+    }
     registerTable(structTypeTable);
 
     // Register "STRUCT.T_10" view.
     // Same columns as "STRUCT.T",
     // but "F0.C0" is set to 10 by default,
     // which is the equivalent of:
-    //   SELECT K0, C1, F0, F1, F2
+    //   SELECT *
     //   FROM T
     //   WHERE F0.C0 = 10
     MockTable struct10View = new MockViewTable(this,
         structTypeSchema.getCatalogName(),
         structTypeSchema.name, "T_10", false, 20,
-        structTypeTable, null, ImmutableIntList.of(0, 1, 2, 3, 4)) {
+        structTypeTable, ImmutableIntList.of(0, 1, 2, 3, 4, 5, 6, 7, 8),
+        structTypeTableResolver) {
 
       @Override public RexNode getConstraint(RexBuilder rexBuilder,
           RelDataType tableRowType) {
-        final RelDataTypeField f0Field =
-            tableRowType.getFieldList().get(2);
         final RelDataTypeField c0Field =
-            f0Field.getType().getFieldList().get(0);
+            tableRowType.getFieldList().get(4);
         return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
-            rexBuilder.makeFieldAccess(
-                rexBuilder.makeInputRef(f0Field.getType(),
-                    f0Field.getIndex()),
+            rexBuilder.makeInputRef(c0Field.getType(),
                 c0Field.getIndex()),
             rexBuilder.makeExactLiteral(BigDecimal.valueOf(10L),
                 c0Field.getType()));
       }
     };
     structTypeSchema.addTable(Util.last(struct10View.getQualifiedName()));
-    struct10View.addColumn("K0", varchar20Type);
-    struct10View.addColumn("C1", varchar20Type);
-    struct10View.addColumn("F0", f0Type);
-    struct10View.addColumn("F1", f1Type);
-    struct10View.addColumn("F2", f2Type);
+    for (CompoundNameColumn column : columns) {
+      struct10View.addColumn(column.getName(), column.type);
+    }
     registerTable(struct10View);
-
-    // TODO Remove this table and use STRUCT.T instead after
-    // CALCITE-1425 is done.
-    MockTable simpleTable = new MockTable(this,
-        structTypeSchema.getCatalogName(), structTypeSchema.name,
-        "SIMPLE", false, 100,
-        ImmutableList.<List<String>>of(
-            ImmutableList.of("K2"),
-            ImmutableList.of("K0")));
-    simpleTable.addColumn("K0", varchar20Type);
-    simpleTable.addColumn("K1", intTypeNull);
-    simpleTable.addColumn("K2", timestampType);
-    registerTable(simpleTable);
     return this;
   }
 
@@ -624,6 +592,12 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Column resolver*/
+  public interface ColumnResolver {
+    List<Pair<RelDataTypeField, List<String>>> resolveColumn(
+        RelDataType rowType, RelDataTypeFactory typeFactory, List<String> names);
+  }
+
   /** Mock schema. */
   public static class MockSchema {
     private final List<String> tableNames = Lists.newArrayList();
@@ -661,42 +635,73 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     protected final List<String> names;
     private final Set<String> monotonicColumnSet = Sets.newHashSet();
     private StructKind kind = StructKind.FULLY_QUALIFIED;
-    protected final List<List<String>> customExpansionList;
+    protected final ColumnResolver resolver;
 
     public MockTable(MockCatalogReader catalogReader, String catalogName,
         String schemaName, String name, boolean stream, double rowCount,
-        List<List<String>> customExpansionList) {
+        ColumnResolver resolver) {
       this.catalogReader = catalogReader;
       this.stream = stream;
       this.rowCount = rowCount;
       this.names = ImmutableList.of(catalogName, schemaName, name);
-      this.customExpansionList = customExpansionList;
+      this.resolver = resolver;
+    }
+
+    /** Implementation of AbstractModifiableTable. */
+    private class ModifiableTable extends JdbcTest.AbstractModifiableTable {
+
+      protected ModifiableTable(String tableName) {
+        super(tableName);
+      }
+
+      @Override public RelDataType
+      getRowType(RelDataTypeFactory typeFactory) {
+        return typeFactory.createStructType(rowType.getFieldList());
+      }
+
+      @Override public Collection getModifiableCollection() {
+        return null;
+      }
+
+      @Override public <E> Queryable<E>
+      asQueryable(QueryProvider queryProvider, SchemaPlus schema,
+          String tableName) {
+        return null;
+      }
+
+      @Override public Type getElementType() {
+        return null;
+      }
+
+      @Override public Expression getExpression(SchemaPlus schema,
+          String tableName, Class clazz) {
+        return null;
+      }
     }
 
     /**
-     * Subclass of AbstractModifiableTable that also implements
-     * CustomExpansionTable.
+     * Subclass of ModifiableTable that also implements
+     * CustomColumnResolvingTable.
      */
-    private abstract class AbstractModifiableTableWithCustomExpansion
-        extends JdbcTest.AbstractModifiableTable
-        implements CustomExpansionTable {
+    private class ModifiableTableWithCustomColumnResolving
+        extends ModifiableTable implements CustomColumnResolvingTable {
 
-      protected AbstractModifiableTableWithCustomExpansion(String tableName) {
+      protected ModifiableTableWithCustomColumnResolving(String tableName) {
         super(tableName);
       }
-    }
 
-    public static MockTable create(MockCatalogReader catalogReader,
-        MockSchema schema, String name, boolean stream, double rowCount) {
-      return create(catalogReader, schema, name, stream, rowCount, null);
+      @Override public List<Pair<RelDataTypeField, List<String>>> resolveColumn(
+          RelDataType rowType, RelDataTypeFactory typeFactory, List<String> names) {
+        return resolver.resolveColumn(rowType, typeFactory, names);
+      }
+
     }
 
     public static MockTable create(MockCatalogReader catalogReader,
-        MockSchema schema, String name, boolean stream, double rowCount,
-        List<List<String>> customExpansionList) {
+        MockSchema schema, String name, boolean stream, double rowCount) {
       MockTable table =
           new MockTable(catalogReader, schema.getCatalogName(), schema.name,
-              name, stream, rowCount, customExpansionList);
+              name, stream, rowCount, null);
       schema.addTable(name);
       return table;
     }
@@ -706,36 +711,10 @@ public class MockCatalogReader implements Prepare.CatalogReader {
         return clazz.cast(this);
       }
       if (clazz.isAssignableFrom(Table.class)) {
-        return clazz.cast(
-            new AbstractModifiableTableWithCustomExpansion(Util.last(names)) {
-              @Override public RelDataType
-              getRowType(RelDataTypeFactory typeFactory) {
-                return typeFactory.createStructType(rowType.getFieldList());
-              }
-
-              @Override public Collection getModifiableCollection() {
-                return null;
-              }
-
-              @Override public <E> Queryable<E>
-              asQueryable(QueryProvider queryProvider, SchemaPlus schema,
-                  String tableName) {
-                return null;
-              }
-
-              @Override public Type getElementType() {
-                return null;
-              }
-
-              @Override public Expression getExpression(SchemaPlus schema,
-                  String tableName, Class clazz) {
-                return null;
-              }
-
-              @Override public List<List<String>> getCustomStarExpansion() {
-                return customExpansionList;
-              }
-            });
+        final Table table = resolver == null
+            ? new ModifiableTable(Util.last(names))
+                : new ModifiableTableWithCustomColumnResolving(Util.last(names));
+        return clazz.cast(table);
       }
       return null;
     }
@@ -807,7 +786,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
     public RelOptTable extend(List<RelDataTypeField> extendedFields) {
       final MockTable table = new MockTable(catalogReader, names.get(0),
-          names.get(1), names.get(2), stream, rowCount, customExpansionList);
+          names.get(1), names.get(2), stream, rowCount, resolver);
       table.columnList.addAll(columnList);
       table.columnList.addAll(extendedFields);
       table.onRegister(catalogReader.typeFactory);
@@ -834,15 +813,69 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
     MockViewTable(MockCatalogReader catalogReader, String catalogName,
         String schemaName, String name, boolean stream, double rowCount,
-        MockTable fromTable, List<List<String>> customExpansionList,
-        ImmutableIntList mapping) {
-      super(catalogReader, catalogName, schemaName, name, stream, rowCount,
-          customExpansionList);
+        MockTable fromTable, ImmutableIntList mapping, ColumnResolver resolver) {
+      super(catalogReader, catalogName,
+          schemaName, name, stream, rowCount, resolver);
       this.fromTable = fromTable;
       this.table = fromTable.unwrap(Table.class);
       this.mapping = mapping;
     }
 
+    /** Implementation of AbstractModifiableView. */
+    private class ModifiableView extends JdbcTest.AbstractModifiableView {
+
+      @Override public Table getTable() {
+        return fromTable.unwrap(Table.class);
+      }
+
+      @Override public Path getTablePath() {
+        final ImmutableList.Builder<Pair<String, Schema>> builder =
+            ImmutableList.builder();
+        for (String name : fromTable.names) {
+          builder.add(Pair.<String, Schema>of(name, null));
+        }
+        return Schemas.path(builder.build());
+      }
+
+      @Override public ImmutableIntList getColumnMapping() {
+        return mapping;
+      }
+
+      @Override public RexNode getConstraint(RexBuilder rexBuilder,
+          RelDataType tableRowType) {
+        return MockViewTable.this.getConstraint(rexBuilder, tableRowType);
+      }
+
+      @Override public RelDataType
+      getRowType(final RelDataTypeFactory typeFactory) {
+        return typeFactory.createStructType(
+            new AbstractList<Map.Entry<String, RelDataType>>() {
+              @Override public Map.Entry<String, RelDataType>
+              get(int index) {
+                return table.getRowType(typeFactory).getFieldList()
+                    .get(mapping.get(index));
+              }
+
+              @Override public int size() {
+                return mapping.size();
+              }
+            });
+      }
+    }
+
+    /**
+     * Subclass of ModifiableView that also implements
+     * CustomColumnResolvingTable.
+     */
+    private class ModifiableViewWithCustomColumnResolving
+      extends ModifiableView implements CustomColumnResolvingTable {
+
+      @Override public List<Pair<RelDataTypeField, List<String>>> resolveColumn(
+          RelDataType rowType, RelDataTypeFactory typeFactory, List<String> names) {
+        return resolver.resolveColumn(rowType, typeFactory, names);
+      }
+    }
+
     protected abstract RexNode getConstraint(RexBuilder rexBuilder,
         RelDataType tableRowType);
 
@@ -876,46 +909,10 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
     @Override public <T> T unwrap(Class<T> clazz) {
       if (clazz.isAssignableFrom(ModifiableView.class)) {
-        return clazz.cast(
-            new JdbcTest.AbstractModifiableView() {
-              @Override public Table getTable() {
-                return fromTable.unwrap(Table.class);
-              }
-
-              @Override public Path getTablePath() {
-                final ImmutableList.Builder<Pair<String, Schema>> builder =
-                    ImmutableList.builder();
-                for (String name : fromTable.names) {
-                  builder.add(Pair.<String, Schema>of(name, null));
-                }
-                return Schemas.path(builder.build());
-              }
-
-              @Override public ImmutableIntList getColumnMapping() {
-                return mapping;
-              }
-
-              @Override public RexNode getConstraint(RexBuilder rexBuilder,
-                  RelDataType tableRowType) {
-                return MockViewTable.this.getConstraint(rexBuilder, tableRowType);
-              }
-
-              @Override public RelDataType
-              getRowType(final RelDataTypeFactory typeFactory) {
-                return typeFactory.createStructType(
-                    new AbstractList<Map.Entry<String, RelDataType>>() {
-                      @Override public Map.Entry<String, RelDataType>
-                      get(int index) {
-                        return table.getRowType(typeFactory).getFieldList()
-                            .get(mapping.get(index));
-                      }
-
-                      @Override public int size() {
-                        return mapping.size();
-                      }
-                    });
-              }
-            });
+        ModifiableView view = resolver == null
+            ? new ModifiableView()
+                : new ModifiableViewWithCustomColumnResolving();
+        return clazz.cast(view);
       }
       return super.unwrap(clazz);
     }
@@ -1047,6 +1044,150 @@ public class MockCatalogReader implements Prepare.CatalogReader {
       return delegate.getComparability();
     }
   }
+
+  /** Column having names with multiple parts. */
+  private static final class CompoundNameColumn {
+    final String first;
+    final String second;
+    final RelDataType type;
+
+    CompoundNameColumn(String first, String second, RelDataType type) {
+      this.first = first;
+      this.second = second;
+      this.type = type;
+    }
+
+    String getName() {
+      return (first.isEmpty() ? "" : ("\"" + first + "\"."))
+          + ("\"" + second + "\"");
+    }
+  }
+
+  /** ColumnResolver implementation that resolves CompoundNameColumn by simulating
+   *  Phoenix behaviors. */
+  private static final class CompoundNameColumnResolver implements ColumnResolver {
+    private final Map<String, Integer> nameMap = Maps.newHashMap();
+    private final Map<String, Map<String, Integer>> groupMap = Maps.newHashMap();
+    private final String defaultColumnGroup;
+
+    CompoundNameColumnResolver(
+        List<CompoundNameColumn> columns, String defaultColumnGroup) {
+      this.defaultColumnGroup = defaultColumnGroup;
+      for (Ord<CompoundNameColumn> column : Ord.zip(columns)) {
+        nameMap.put(column.e.getName(), column.i);
+        Map<String, Integer> subMap = groupMap.get(column.e.first);
+        if (subMap == null) {
+          subMap = Maps.newHashMap();
+          groupMap.put(column.e.first, subMap);
+        }
+        subMap.put(column.e.second, column.i);
+      }
+    }
+
+    @Override public List<Pair<RelDataTypeField, List<String>>> resolveColumn(
+        RelDataType rowType, RelDataTypeFactory typeFactory, List<String> names) {
+      List<Pair<RelDataTypeField, List<String>>> ret = new ArrayList<>();
+      if (names.size() >= 2) {
+        Map<String, Integer> subMap = groupMap.get(names.get(0));
+        if (subMap != null) {
+          Integer index = subMap.get(names.get(1));
+          if (index != null) {
+            ret.add(
+                new Pair<RelDataTypeField, List<String>>(
+                    rowType.getFieldList().get(index),
+                    names.subList(2, names.size())));
+          }
+        }
+      }
+
+      final String columnName = names.get(0);
+      final List<String> remainder = names.subList(1, names.size());
+      Integer index = nameMap.get(columnName);
+      if (index != null) {
+        ret.add(
+            new Pair<RelDataTypeField, List<String>>(
+                rowType.getFieldList().get(index), remainder));
+        return ret;
+      }
+
+      final List<String> priorityGroups = Arrays.asList("", defaultColumnGroup);
+      for (String group : priorityGroups) {
+        Map<String, Integer> subMap = groupMap.get(group);
+        if (subMap != null) {
+          index = subMap.get(columnName);
+          if (index != null) {
+            ret.add(
+                new Pair<RelDataTypeField, List<String>>(
+                    rowType.getFieldList().get(index), remainder));
+            return ret;
+          }
+        }
+      }
+      for (Map.Entry<String, Map<String, Integer>> entry : groupMap.entrySet()) {
+        if (priorityGroups.contains(entry.getKey())) {
+          continue;
+        }
+        index = entry.getValue().get(columnName);
+        if (index != null) {
+          ret.add(
+              new Pair<RelDataTypeField, List<String>>(
+                  rowType.getFieldList().get(index), remainder));
+        }
+      }
+
+      if (ret.isEmpty() && names.size() == 1) {
+        Map<String, Integer> subMap = groupMap.get(columnName);
+        if (subMap != null) {
+          List<Map.Entry<String, Integer>> entries =
+              new ArrayList<>(subMap.entrySet());
+          Collections.sort(
+              entries,
+              new Comparator<Map.Entry<String, Integer>>() {
+                @Override public int compare(
+                    Entry<String, Integer> o1, Entry<String, Integer> o2) {
+                  return o1.getValue() - o2.getValue();
+                }
+              });
+          ret.add(
+              new Pair<RelDataTypeField, List<String>>(
+                  new RelDataTypeFieldImpl(
+                      columnName, -1,
+                      createStructType(
+                          rowType,
+                          typeFactory,
+                          entries)),
+                  remainder));
+        }
+      }
+
+      return ret;
+    }
+
+    private static RelDataType createStructType(
+        final RelDataType rowType,
+        RelDataTypeFactory typeFactory,
+        final List<Map.Entry<String, Integer>> entries) {
+      return typeFactory.createStructType(
+          StructKind.PEEK_FIELDS,
+          new AbstractList<RelDataType>() {
+            @Override public RelDataType get(int index) {
+              final int i = entries.get(index).getValue();
+              return rowType.getFieldList().get(i).getType();
+            }
+            @Override public int size() {
+              return entries.size();
+            }
+          },
+          new AbstractList<String>() {
+            @Override public String get(int index) {
+              return entries.get(index).getKey();
+            }
+            @Override public int size() {
+              return entries.size();
+            }
+          });
+    }
+  }
 }
 
 // End MockCatalogReader.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 62ffe4f..74e2b03 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -2744,7 +2744,7 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkSubQuery(sql).check();
   }
 
-  @Test public void testStructTypeInNonCorrelatedSubQuery() {
+  @Test public void testCustomColumnResolvingInNonCorrelatedSubQuery() {
     final String sql = "select *\n"
         + "from struct.t t1\n"
         + "where c0 in (\n"
@@ -2758,7 +2758,7 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkPlanning(tester, null, new HepPlanner(program), sql);
   }
 
-  @Test public void testStructTypeInCorrelatedSubQuery() {
+  @Test public void testCustomColumnResolvingInCorrelatedSubQuery() {
     final String sql = "select *\n"
         + "from struct.t t1\n"
         + "where c0 = (\n"
@@ -2772,7 +2772,7 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkPlanning(tester, null, new HepPlanner(program), sql);
   }
 
-  @Test public void testStructTypeInCorrelatedSubQuery2() {
+  @Test public void testCustomColumnResolvingInCorrelatedSubQuery2() {
     final String sql = "select *\n"
         + "from struct.t t1\n"
         + "where c0 in (\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/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 d7fa30c..f3e5a24 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -1514,6 +1514,26 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
+  @Test public void testInsertWithCustomColumnResolving() {
+    final String sql = "insert into struct.t values (?, ?, ?, ?, ?, ?, ?, ?, ?)";
+    sql(sql).ok();
+  }
+
+  @Test public void testInsertWithCustomColumnResolving2() {
+    final String sql = "insert into struct.t (c0, c2, c1) values (?, ?, ?)";
+    sql(sql).ok();
+  }
+
+  @Test public void testInsertViewWithCustomColumnResolving() {
+    final String sql = "insert into struct.t_10 (c0, c2, c1) values (?, ?, ?)";
+    sql(sql).ok();
+  }
+
+  @Test public void testUpdateWithCustomColumnResolving() {
+    final String sql = "update struct.t set c0 = c0 + 1";
+    sql(sql).ok();
+  }
+
   /**
    * Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-695">[CALCITE-695]
@@ -1757,38 +1777,27 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).decorrelate(true).ok();
   }
 
-  @Test public void testStructType() {
+  @Test public void testCustomColumnResolving() {
     final String sql = "select k0 from struct.t";
     sql(sql).ok();
   }
 
-  @Test public void testStructType2() {
+  @Test public void testCustomColumnResolving2() {
     final String sql = "select c2 from struct.t";
     sql(sql).ok();
   }
 
-  @Test public void testStructType3() {
+  @Test public void testCustomColumnResolving3() {
     final String sql = "select f1.c2 from struct.t";
     sql(sql).ok();
   }
 
-  @Test public void testStructType4() {
-    final String sql = "select f1 from struct.t";
-    sql(sql).ok();
-  }
-
-  @Test public void testStructTypeWithSelectStar() {
+  @Test public void testCustomColumnResolvingWithSelectStar() {
     final String sql = "select * from struct.t";
     sql(sql).ok();
   }
 
-  @Test public void testStructTypeWithSelectStar2() {
-    final String sql = "select * from struct.t t1 join struct.t t2\n"
-        + "on t1.k0 = t2.k0";
-    sql(sql).ok();
-  }
-
-  @Test public void testStructTypeWithSelectFieldNameDotStar() {
+  @Test public void testCustomColumnResolvingWithSelectFieldNameDotStar() {
     final String sql = "select f1.* from struct.t";
     sql(sql).ok();
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/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 83125ba..881da12 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -7972,10 +7972,24 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     sql(sql2).ok().bindType(expected2);
   }
 
-  @Test public void testInsertBindWithCustomStarExpansion() {
-    sql("insert into struct.simple values (?, ?)")
-        .ok()
-        .bindType("RecordType(TIMESTAMP(0) ?0, VARCHAR(20) ?1)");
+  @Test public void testInsertBindWithCustomColumnResolving() {
+    final String sql = "insert into struct.t\n"
+        + "values (?, ?, ?, ?, ?, ?, ?, ?, ?)";
+    final String expected = "RecordType(VARCHAR(20) ?0, VARCHAR(20) ?1,"
+        + " INTEGER ?2, BOOLEAN ?3, INTEGER ?4, INTEGER ?5, INTEGER ?6,"
+        + " INTEGER ?7, INTEGER ?8)";
+    sql(sql).ok().bindType(expected);
+
+    final String sql2 =
+        "insert into struct.t (c0, c2, c1) values (?, ?, ?)";
+    final String expected2 =
+        "RecordType(INTEGER ?0, INTEGER ?1, VARCHAR(20) ?2)";
+    sql(sql2).ok().bindType(expected2);
+
+    sql("insert into struct.t (c0, ^c4^, c1) values (?, ?, ?)")
+        .fails("Unknown target column 'C4'");
+    sql("insert into struct.t (^a0^, c2, c1) values (?, ?, ?)")
+        .fails("Unknown target column 'A0'");
   }
 
   @Test public void testUpdateBind() {
@@ -8223,15 +8237,15 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // (To debug individual statements, paste them into this method.)
   }
 
-  @Test public void testStructType() {
-    checkStructType("T");
+  @Test public void testCustomColumnResolving() {
+    checkCustomColumnResolving("T");
   }
 
-  @Test public void testStructTypeWithView() {
-    checkStructType("T_10");
+  @Test public void testCustomColumnResolvingWithView() {
+    checkCustomColumnResolving("T_10");
   }
 
-  private void checkStructType(String table) {
+  private void checkCustomColumnResolving(String table) {
     // Table STRUCT.T is defined as: (
     //   K0 VARCHAR(20) NOT NULL,
     //   C1 VARCHAR(20) NOT NULL,
@@ -8354,26 +8368,21 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     // Resolve struct type F1 with wildcard.
     sql("select f1.* from struct." + table)
-        .type("RecordType(INTEGER C0, INTEGER NOT NULL C2,"
-            + " INTEGER NOT NULL A0) NOT NULL");
+        .type("RecordType(INTEGER NOT NULL A0, INTEGER C0,"
+            + " INTEGER NOT NULL C2) NOT NULL");
 
     // Resolve struct type F1 with wildcard.
     sql("select " + table + ".f1.* from struct." + table)
-        .type("RecordType(INTEGER C0, INTEGER NOT NULL C2,"
-            + " INTEGER NOT NULL A0) NOT NULL");
+        .type("RecordType(INTEGER NOT NULL A0, INTEGER C0,"
+            + " INTEGER NOT NULL C2) NOT NULL");
 
     // Fail non-existent column B0.
     sql("select ^b0^ from struct." + table)
         .fails("Column 'B0' not found in any table");
 
-    // 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." + table)
-        .type("RecordType(RecordType:peek(INTEGER C0, INTEGER NOT NULL C2,"
-            + " INTEGER NOT NULL A0) NOT NULL F1) NOT NULL");
+    // A column family can only be referenced with a star expansion.
+    sql("select ^f1^ from struct." + table)
+        .fails("Column 'F1' not found in any table");
 
     // If we fail to find a column, give an error based on the shortest prefix
     // that fails.

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index e1f0b8f..e0836dd 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -1300,7 +1300,7 @@ LogicalProject(EMPNO=[$0])
 ]]>
         </Resource>
     </TestCase>
-    <TestCase name="testStructTypeInNonCorrelatedSubQuery">
+    <TestCase name="testCustomColumnResolvingInNonCorrelatedSubQuery">
         <Resource name="sql">
             <![CDATA[select *
 from struct.t t1
@@ -1309,31 +1309,27 @@ where c0 in (
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(K0=[$0], C1=[$1], A0=[$6], A00=[$8], C0=[$2], C00=[$4], C10=[$3], C2=[$5], C3=[$7])
-  LogicalFilter(condition=[IN($2, {
-LogicalProject(C0=[$4])
-  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]])
+LogicalProject("K0"=[$0], "C1"=[$1], "F1"."A0"=[$2], "F2"."A0"=[$3], "F0"."C0"=[$4], "F1"."C0"=[$5], "F0"."C1"=[$6], "F1"."C2"=[$7], "F2"."C3"=[$8])
+  LogicalFilter(condition=[IN($4, {
+LogicalProject(C0=[$5])
+  LogicalTableScan(table=[[CATALOG, STRUCT, T]])
 })])
-    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]])
+    LogicalTableScan(table=[[CATALOG, STRUCT, T]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(K0=[$0], C1=[$1], A0=[$6], A00=[$8], C0=[$2], C00=[$4], C10=[$3], C2=[$5], C3=[$7])
-  LogicalProject(K0=[$0], C1=[$1], C0=[$2], C13=[$3], C04=[$4], C2=[$5], A0=[$6], C3=[$7], A08=[$8])
-    LogicalJoin(condition=[=($2, $9)], joinType=[inner])
-      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]])
+LogicalProject("K0"=[$0], "C1"=[$1], "F1"."A0"=[$2], "F2"."A0"=[$3], "F0"."C0"=[$4], "F1"."C0"=[$5], "F0"."C1"=[$6], "F1"."C2"=[$7], "F2"."C3"=[$8])
+  LogicalProject("K0"=[$0], "C1"=[$1], "F1"."A0"=[$2], "F2"."A0"=[$3], "F0"."C0"=[$4], "F1"."C0"=[$5], "F0"."C1"=[$6], "F1"."C2"=[$7], "F2"."C3"=[$8])
+    LogicalJoin(condition=[=($4, $9)], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, STRUCT, T]])
       LogicalAggregate(group=[{0}])
-        LogicalProject(C0=[$4])
-          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]])
+        LogicalProject(C0=[$5])
+          LogicalTableScan(table=[[CATALOG, STRUCT, T]])
 ]]>
         </Resource>
     </TestCase>
-    <TestCase name="testStructTypeInCorrelatedSubQuery">
+    <TestCase name="testCustomColumnResolvingInCorrelatedSubQuery">
         <Resource name="sql">
             <![CDATA[select *
 from struct.t t1
@@ -1342,35 +1338,31 @@ where c0 = (
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(K0=[$0], C1=[$1], A0=[$6], A00=[$8], C0=[$2], C00=[$4], C10=[$3], C2=[$5], C3=[$7])
-  LogicalFilter(condition=[=($2, $SCALAR_QUERY({
+LogicalProject("K0"=[$0], "C1"=[$1], "F1"."A0"=[$2], "F2"."A0"=[$3], "F0"."C0"=[$4], "F1"."C0"=[$5], "F0"."C1"=[$6], "F1"."C2"=[$7], "F2"."C3"=[$8])
+  LogicalFilter(condition=[=($4, $SCALAR_QUERY({
 LogicalAggregate(group=[{}], EXPR$0=[MAX($0)])
-  LogicalProject($f0=[$4])
-    LogicalFilter(condition=[=($cor0.K0_0, $0)])
-      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]])
-}))], variablesSet=[[$cor0]])
-    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])
+  LogicalProject("F1"."C0"=[$5])
+    LogicalFilter(condition=[=($cor0."K0", $0)])
       LogicalTableScan(table=[[CATALOG, STRUCT, T]])
+}))], variablesSet=[[$cor0]])
+    LogicalTableScan(table=[[CATALOG, STRUCT, T]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(K0=[$0], C1=[$1], A0=[$6], A00=[$8], C0=[$2], C00=[$4], C10=[$3], C2=[$5], C3=[$7])
-  LogicalProject(K0=[$0], C1=[$1], C0=[$2], C13=[$3], C04=[$4], C2=[$5], A0=[$6], C3=[$7], A08=[$8])
-    LogicalFilter(condition=[=($2, $9)])
+LogicalProject("K0"=[$0], "C1"=[$1], "F1"."A0"=[$2], "F2"."A0"=[$3], "F0"."C0"=[$4], "F1"."C0"=[$5], "F0"."C1"=[$6], "F1"."C2"=[$7], "F2"."C3"=[$8])
+  LogicalProject("K0"=[$0], "C1"=[$1], "F1"."A0"=[$2], "F2"."A0"=[$3], "F0"."C0"=[$4], "F1"."C0"=[$5], "F0"."C1"=[$6], "F1"."C2"=[$7], "F2"."C3"=[$8])
+    LogicalFilter(condition=[=($4, $9)])
       LogicalCorrelate(correlation=[$cor0], joinType=[LEFT], requiredColumns=[{0}])
-        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]])
+        LogicalTableScan(table=[[CATALOG, STRUCT, T]])
         LogicalAggregate(group=[{}], EXPR$0=[MAX($0)])
-          LogicalProject($f0=[$4])
-            LogicalFilter(condition=[=($cor0.K0_0, $0)])
-              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]])
+          LogicalProject("F1"."C0"=[$5])
+            LogicalFilter(condition=[=($cor0."K0", $0)])
+              LogicalTableScan(table=[[CATALOG, STRUCT, T]])
 ]]>
         </Resource>
     </TestCase>
-    <TestCase name="testStructTypeInCorrelatedSubQuery2">
+    <TestCase name="testCustomColumnResolvingInCorrelatedSubQuery2">
         <Resource name="sql">
             <![CDATA[select *
 from struct.t t1
@@ -1379,30 +1371,26 @@ where c0 in (
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(K0=[$0], C1=[$1], A0=[$6], A00=[$8], C0=[$2], C00=[$4], C10=[$3], C2=[$5], C3=[$7])
-  LogicalFilter(condition=[IN($2, {
-LogicalProject(C0=[$4])
-  LogicalFilter(condition=[=($cor0.C2_5, $5)])
-    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]])
+LogicalProject("K0"=[$0], "C1"=[$1], "F1"."A0"=[$2], "F2"."A0"=[$3], "F0"."C0"=[$4], "F1"."C0"=[$5], "F0"."C1"=[$6], "F1"."C2"=[$7], "F2"."C3"=[$8])
+  LogicalFilter(condition=[IN($4, {
+LogicalProject(C0=[$5])
+  LogicalFilter(condition=[=($cor0."F1"."C2", $7)])
+    LogicalTableScan(table=[[CATALOG, STRUCT, T]])
 })], variablesSet=[[$cor0]])
-    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]])
+    LogicalTableScan(table=[[CATALOG, STRUCT, T]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(K0=[$0], C1=[$1], A0=[$6], A00=[$8], C0=[$2], C00=[$4], C10=[$3], C2=[$5], C3=[$7])
-  LogicalProject(K0=[$0], C1=[$1], C0=[$2], C13=[$3], C04=[$4], C2=[$5], A0=[$6], C3=[$7], A08=[$8])
-    LogicalFilter(condition=[=($2, $9)])
-      LogicalCorrelate(correlation=[$cor0], joinType=[INNER], requiredColumns=[{5}])
-        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]])
+LogicalProject("K0"=[$0], "C1"=[$1], "F1"."A0"=[$2], "F2"."A0"=[$3], "F0"."C0"=[$4], "F1"."C0"=[$5], "F0"."C1"=[$6], "F1"."C2"=[$7], "F2"."C3"=[$8])
+  LogicalProject("K0"=[$0], "C1"=[$1], "F1"."A0"=[$2], "F2"."A0"=[$3], "F0"."C0"=[$4], "F1"."C0"=[$5], "F0"."C1"=[$6], "F1"."C2"=[$7], "F2"."C3"=[$8])
+    LogicalFilter(condition=[=($4, $9)])
+      LogicalCorrelate(correlation=[$cor0], joinType=[INNER], requiredColumns=[{7}])
+        LogicalTableScan(table=[[CATALOG, STRUCT, T]])
         LogicalAggregate(group=[{0}])
-          LogicalProject(C0=[$4])
-            LogicalFilter(condition=[=($cor0.C2_5, $5)])
-              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]])
+          LogicalProject(C0=[$5])
+            LogicalFilter(condition=[=($cor0."F1"."C2", $7)])
+              LogicalTableScan(table=[[CATALOG, STRUCT, T]])
 ]]>
         </Resource>
     </TestCase>

http://git-wip-us.apache.org/repos/asf/calcite/blob/46654ad2/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 4315a43..9880688 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -2613,92 +2613,110 @@ LogicalTableModify(table=[[CATALOG, SALES, EMP]], operation=[INSERT], updateColu
 ]]>
         </Resource>
     </TestCase>
-    <TestCase name="testStructType">
+    <TestCase name="testInsertWithCustomColumnResolving">
         <Resource name="sql">
-            <![CDATA[select k0 from struct.t]]>
+            <![CDATA[insert into struct.t values (?, ?, ?, ?, ?, ?, ?, ?, ?)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(K0=[$0])
-  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]])
+LogicalTableModify(table=[[CATALOG, STRUCT, T]], operation=[INSERT], updateColumnList=[[]], flattened=[true])
+  LogicalProject("K0"=[?0], "C1"=[?1], "F1"."A0"=[?2], "F2"."A0"=[?3], "F0"."C0"=[?4], "F1"."C0"=[?5], "F0"."C1"=[?6], "F1"."C2"=[?7], "F2"."C3"=[?8])
+    LogicalValues(tuples=[[{ 0 }]])
 ]]>
         </Resource>
     </TestCase>
-    <TestCase name="testStructType2">
+    <TestCase name="testInsertWithCustomColumnResolving2">
         <Resource name="sql">
-            <![CDATA[select c2 from struct.t]]>
+            <![CDATA[insert into struct.t (c0, c2, c1) values (?, ?, ?)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(C2=[$5])
-  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]])
+LogicalTableModify(table=[[CATALOG, STRUCT, T]], operation=[INSERT], updateColumnList=[[]], flattened=[true])
+  LogicalProject("K0"=[null], "C1"=[$2], "F1"."A0"=[null], "F2"."A0"=[null], "F0"."C0"=[$0], "F1"."C0"=[null], "F0"."C1"=[null], "F1"."C2"=[$1], "F2"."C3"=[null])
+    LogicalProject(EXPR$0=[?0], EXPR$1=[?1], EXPR$2=[?2])
+      LogicalValues(tuples=[[{ 0 }]])
 ]]>
         </Resource>
     </TestCase>
-    <TestCase name="testStructType3">
+    <TestCase name="testInsertViewWithCustomColumnResolving">
         <Resource name="sql">
-            <![CDATA[select f1.c2 from struct.t]]>
+            <![CDATA[insert into struct.t_10 (c0, c2, c1) values (?, ?, ?)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalTableModify(table=[[CATALOG, STRUCT, T]], operation=[INSERT], updateColumnList=[[]], flattened=[true])
+  LogicalFilter(condition=[=($4, 10)])
+    LogicalProject("K0"=[CAST($0):VARCHAR(20) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL], "C1"=[CAST($1):VARCHAR(20) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL], "F1"."A0"=[CAST($2):INTEGER NOT NULL], "F2"."A0"=[CAST($3):BOOLEAN NOT NULL], "F0"."C0"=[CAST($4):INTEGER NOT NULL], "F1"."C0"=[$5], "F0"."C1"=[CAST($6):INTEGER NOT NULL], "F1"."C2"=[CAST($7):INTEGER NOT NULL], "F2"."C3"=[CAST($8):INTEGER NOT NULL])
+      LogicalProject("K0"=[null], "C1"=[$2], "F1"."A0"=[null], "F2"."A0"=[null], "F0"."C0"=[$0], "F1"."C0"=[null], "F0"."C1"=[null], "F1"."C2"=[$1], "F2"."C3"=[null])
+        LogicalProject(EXPR$0=[?0], EXPR$1=[?1], EXPR$2=[?2])
+          LogicalValues(tuples=[[{ 0 }]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testUpdateWithCustomColumnResolving">
+        <Resource name="sql">
+            <![CDATA[update struct.t set c0 = c0 + 1]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(C2=[$5])
-  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])
+LogicalTableModify(table=[[CATALOG, STRUCT, T]], operation=[UPDATE], updateColumnList=[["F0"."C0"]], flattened=[true])
+  LogicalProject("K0"=[$0], "C1"=[$1], "F1"."A0"=[$2], "F2"."A0"=[$3], "F0"."C0"=[$4], "F1"."C0"=[$5], "F0"."C1"=[$6], "F1"."C2"=[$7], "F2"."C3"=[$8], EXPR$0=[+($4, 1)])
     LogicalTableScan(table=[[CATALOG, STRUCT, T]])
 ]]>
         </Resource>
     </TestCase>
-    <TestCase name="testStructType4">
+    <TestCase name="testCustomColumnResolving">
         <Resource name="sql">
-            <![CDATA[select f1 from struct.t]]>
+            <![CDATA[select k0 from struct.t]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(F1=[$0])
-  LogicalProject(F1=[$4], F11=[$5], F12=[$6])
-    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]])
+LogicalProject(K0=[$0])
+  LogicalTableScan(table=[[CATALOG, STRUCT, T]])
 ]]>
         </Resource>
     </TestCase>
-    <TestCase name="testStructTypeWithSelectStar">
+    <TestCase name="testCustomColumnResolving2">
         <Resource name="sql">
-            <![CDATA[select * from struct.t]]>
+            <![CDATA[select c2 from struct.t]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(K0=[$0], C1=[$1], A0=[$6], A00=[$8], C0=[$2], C00=[$4], C10=[$3], C2=[$5], C3=[$7])
-  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]])
+LogicalProject(C2=[$7])
+  LogicalTableScan(table=[[CATALOG, STRUCT, T]])
 ]]>
         </Resource>
     </TestCase>
-    <TestCase name="testStructTypeWithSelectStar2">
+    <TestCase name="testCustomColumnResolving3">
         <Resource name="sql">
-            <![CDATA[select * from struct.t t1 join struct.t t2
-on t1.k0 = t2.k0]]>
+            <![CDATA[select f1.c2 from struct.t]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(K0=[$0], C1=[$1], A0=[$6], A00=[$8], C0=[$2], C00=[$4], C10=[$3], C2=[$5], C3=[$7], K00=[$9], C11=[$10], A01=[$15], A02=[$17], C01=[$11], C02=[$13], C12=[$12], C20=[$14], C30=[$16])
-  LogicalJoin(condition=[=($0, $9)], joinType=[inner])
-    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]])
-    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]])
+LogicalProject(C2=[$7])
+  LogicalTableScan(table=[[CATALOG, STRUCT, T]])
 ]]>
         </Resource>
     </TestCase>
-    <TestCase name="testStructTypeWithSelectFieldNameDotStar">
+    <TestCase name="testCustomColumnResolvingWithSelectStar">
+        <Resource name="sql">
+            <![CDATA[select * from struct.t]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject("K0"=[$0], "C1"=[$1], "F1"."A0"=[$2], "F2"."A0"=[$3], "F0"."C0"=[$4], "F1"."C0"=[$5], "F0"."C1"=[$6], "F1"."C2"=[$7], "F2"."C3"=[$8])
+  LogicalTableScan(table=[[CATALOG, STRUCT, T]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testCustomColumnResolvingWithSelectFieldNameDotStar">
         <Resource name="sql">
             <![CDATA[select f1.* from struct.t]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(C0=[$4], C2=[$5], A0=[$6])
-  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]])
+LogicalProject("F1"."A0"=[$2], "F1"."C0"=[$5], "F1"."C2"=[$7])
+  LogicalTableScan(table=[[CATALOG, STRUCT, T]])
 ]]>
         </Resource>
     </TestCase>