You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2016/06/15 16:39:33 UTC

[5/5] calcite git commit: [CALCITE-528] When making field names unique, follow type system's case-sensitivity policy (Jacques Nadeau, Jinfeng Ni, Minji Kim)

[CALCITE-528] When making field names unique, follow type system's case-sensitivity policy (Jacques Nadeau, Jinfeng Ni, Minji Kim)

Ensure that uniquify keeps the ordinal position of input names (Jinfeng Ni).

Add tests (Minji Kim).

Fix up (Julian Hyde).

Close apache/calcite#245


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

Branch: refs/heads/master
Commit: acd27fdea5fb54375a999a9221b79a6df166660c
Parents: 0599cdd
Author: Jacques Nadeau <ja...@apache.org>
Authored: Fri Feb 26 19:44:17 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jun 14 18:02:59 2016 -0700

----------------------------------------------------------------------
 .../adapter/enumerable/EnumerableRel.java       |   4 +-
 .../org/apache/calcite/materialize/Lattice.java |   2 +-
 .../org/apache/calcite/plan/RelOptUtil.java     |  27 ++-
 .../calcite/plan/SubstitutionVisitor.java       |  10 +-
 .../calcite/prepare/CalciteCatalogReader.java   |   6 +
 .../java/org/apache/calcite/rel/core/Join.java  | 115 +---------
 .../org/apache/calcite/rel/core/SemiJoin.java   |   3 +-
 .../calcite/rel/logical/LogicalProject.java     |   9 +-
 .../calcite/rel/rules/CalcRelSplitter.java      |   5 +-
 .../rel/rules/JoinProjectTransposeRule.java     |   3 +-
 .../rel/rules/SemiJoinProjectTransposeRule.java |   6 +-
 .../calcite/rel/type/RelDataTypeFactory.java    |  11 +-
 .../calcite/rel/type/RelDataTypeSystem.java     |   4 +
 .../calcite/rel/type/RelDataTypeSystemImpl.java |  23 +-
 .../apache/calcite/rex/RexProgramBuilder.java   |   6 +-
 .../java/org/apache/calcite/rex/RexUtil.java    |  20 +-
 .../java/org/apache/calcite/sql/SqlCall.java    |   5 +-
 .../sql/validate/SqlValidatorCatalogReader.java |   2 +
 .../calcite/sql/validate/SqlValidatorImpl.java  |   5 +-
 .../calcite/sql/validate/SqlValidatorUtil.java  | 226 ++++++++++++++++---
 .../calcite/sql2rel/SqlToRelConverter.java      |   5 +-
 .../sql/validate/LexCaseSensitiveTest.java      | 203 +++++++++++++++++
 .../sql/validate/SqlValidatorUtilTest.java      | 112 +++++++++
 .../org/apache/calcite/test/CalciteSuite.java   |   4 +
 .../apache/calcite/test/MockCatalogReader.java  |   4 +
 25 files changed, 621 insertions(+), 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRel.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRel.java
index 94d13c7..5f9b7d3 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRel.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRel.java
@@ -48,9 +48,7 @@ public interface EnumerableRel
           final RelOptCluster cluster = child.getCluster();
           final RelDataType rowType =
               RexUtil.createStructType(cluster.getTypeFactory(), projects,
-                  fieldNames == null ? null
-                      : SqlValidatorUtil.uniquify(fieldNames,
-                          SqlValidatorUtil.F_SUGGESTER));
+                  fieldNames, SqlValidatorUtil.F_SUGGESTER);
           return EnumerableProject.create(child, projects, rowType);
         }
       };

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/materialize/Lattice.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/Lattice.java b/core/src/main/java/org/apache/calcite/materialize/Lattice.java
index a0f2186..8ebfce9 100644
--- a/core/src/main/java/org/apache/calcite/materialize/Lattice.java
+++ b/core/src/main/java/org/apache/calcite/materialize/Lattice.java
@@ -143,7 +143,7 @@ public class Lattice {
     }
     uniqueColumnNames =
         ImmutableList.copyOf(
-            SqlValidatorUtil.uniquify(Lists.transform(columns, GET_ALIAS)));
+            SqlValidatorUtil.uniquify(Lists.transform(columns, GET_ALIAS), true));
     if (rowCountEstimate == null) {
       // We could improve this when we fix
       // [CALCITE-429] Add statistics SPI for lattice optimization algorithm

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
index d471b52..60e9cb7 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -49,6 +49,7 @@ 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.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexCorrelVariable;
@@ -1541,7 +1542,10 @@ public abstract class RelOptUtil {
       List<Integer> outputProj) {
     RelNode leftRel = inputRels[0];
     RelNode rightRel = inputRels[1];
-    RexBuilder rexBuilder = leftRel.getCluster().getRexBuilder();
+    final RelOptCluster cluster = leftRel.getCluster();
+    final RexBuilder rexBuilder = cluster.getRexBuilder();
+    final RelDataTypeSystem typeSystem =
+        cluster.getTypeFactory().getTypeSystem();
 
     int origLeftInputSize = leftRel.getRowType().getFieldCount();
     int origRightInputSize = rightRel.getRowType().getFieldCount();
@@ -1612,12 +1616,14 @@ public abstract class RelOptUtil {
     // fields
     if (newLeftKeyCount > 0) {
       leftRel = createProject(leftRel, newLeftFields,
-          SqlValidatorUtil.uniquify(newLeftFieldNames));
+          SqlValidatorUtil.uniquify(newLeftFieldNames,
+              typeSystem.isSchemaCaseSensitive()));
     }
 
     if (newRightKeyCount > 0) {
       rightRel = createProject(rightRel, newRightFields,
-          SqlValidatorUtil.uniquify(newRightFieldNames));
+          SqlValidatorUtil.uniquify(newRightFieldNames,
+              typeSystem.isSchemaCaseSensitive()));
     }
 
     inputRels[0] = leftRel;
@@ -2834,19 +2840,12 @@ public abstract class RelOptUtil {
       boolean optimize,
       RelFactories.ProjectFactory projectFactory) {
     final RelOptCluster cluster = child.getCluster();
-    final List<String> fieldNames2 =
-        fieldNames == null
-            ? null
-            : SqlValidatorUtil.uniquify(fieldNames,
-                SqlValidatorUtil.F_SUGGESTER);
+    final RelDataType rowType =
+        RexUtil.createStructType(cluster.getTypeFactory(), exprs,
+            fieldNames, SqlValidatorUtil.F_SUGGESTER);
     if (optimize
         && RexUtil.isIdentity(exprs, child.getRowType())) {
       if (child instanceof Project && fieldNames != null) {
-        final RelDataType rowType =
-            RexUtil.createStructType(
-                cluster.getTypeFactory(),
-                exprs,
-                fieldNames2);
         // Rename columns of child projection if desired field names are given.
         Project childProject = (Project) child;
         child = childProject.copy(childProject.getTraitSet(),
@@ -2854,7 +2853,7 @@ public abstract class RelOptUtil {
       }
       return child;
     }
-    return projectFactory.createProject(child, exprs, fieldNames2);
+    return projectFactory.createProject(child, exprs, rowType.getFieldNames());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
index 0e028f8..dd60c22 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -1678,10 +1678,7 @@ public class SubstitutionVisitor {
         List<String> fieldNameList) {
       final RelDataType rowType =
           RexUtil.createStructType(child.cluster.getTypeFactory(), exprList,
-              fieldNameList == null
-                  ? null
-                  : SqlValidatorUtil.uniquify(fieldNameList,
-                      SqlValidatorUtil.F_SUGGESTER));
+              fieldNameList, SqlValidatorUtil.F_SUGGESTER);
       return of(rowType, child, exprList);
     }
 
@@ -2017,8 +2014,9 @@ public class SubstitutionVisitor {
         Set<CorrelationId> variablesStopped) {
       List<RelDataTypeField> fieldList = Collections.emptyList();
       RelDataType rowType =
-          Join.deriveJoinRowType(left.getRowType(), right.getRowType(),
-              joinType, cluster.getTypeFactory(), null, fieldList);
+          SqlValidatorUtil.deriveJoinRowType(left.getRowType(),
+              right.getRowType(), joinType, cluster.getTypeFactory(), null,
+              fieldList);
       return new MutableJoin(rowType, left, right, condition, joinType,
           variablesStopped);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
index 40299e9..9d8f3ef 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -351,6 +351,12 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
 
   public void registerRules(RelOptPlanner planner) throws Exception {
   }
+
+  @Override public boolean isCaseSensitive() {
+    return caseSensitive;
+  }
+
+
 }
 
 // End CalciteCatalogReader.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/rel/core/Join.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Join.java b/core/src/main/java/org/apache/calcite/rel/core/Join.java
index 4a99f0d..e635cdd 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Join.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Join.java
@@ -32,6 +32,7 @@ import org.apache.calcite.rex.RexChecker;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
@@ -39,9 +40,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
-import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
@@ -214,12 +213,8 @@ public abstract class Join extends BiRel {
   }
 
   @Override protected RelDataType deriveRowType() {
-    return deriveJoinRowType(
-        left.getRowType(),
-        right.getRowType(),
-        joinType,
-        getCluster().getTypeFactory(),
-        null,
+    return SqlValidatorUtil.deriveJoinRowType(left.getRowType(),
+        right.getRowType(), joinType, getCluster().getTypeFactory(), null,
         getSystemFieldList());
   }
 
@@ -246,20 +241,7 @@ public abstract class Join extends BiRel {
     return Collections.emptyList();
   }
 
-  /**
-   * Derives the type of a join relational expression.
-   *
-   * @param leftType        Row type of left input to join
-   * @param rightType       Row type of right input to join
-   * @param joinType        Type of join
-   * @param typeFactory     Type factory
-   * @param fieldNameList   List of names of fields; if null, field names are
-   *                        inherited and made unique
-   * @param systemFieldList List of system fields that will be prefixed to
-   *                        output row type; typically empty but must not be
-   *                        null
-   * @return join type
-   */
+  @Deprecated // to be removed before 2.0
   public static RelDataType deriveJoinRowType(
       RelDataType leftType,
       RelDataType rightType,
@@ -267,98 +249,19 @@ public abstract class Join extends BiRel {
       RelDataTypeFactory typeFactory,
       List<String> fieldNameList,
       List<RelDataTypeField> systemFieldList) {
-    assert systemFieldList != null;
-    switch (joinType) {
-    case LEFT:
-      rightType = typeFactory.createTypeWithNullability(rightType, true);
-      break;
-    case RIGHT:
-      leftType = typeFactory.createTypeWithNullability(leftType, true);
-      break;
-    case FULL:
-      leftType = typeFactory.createTypeWithNullability(leftType, true);
-      rightType = typeFactory.createTypeWithNullability(rightType, true);
-      break;
-    default:
-      break;
-    }
-    return createJoinType(
-        typeFactory, leftType, rightType, fieldNameList, systemFieldList);
+    return SqlValidatorUtil.deriveJoinRowType(leftType, rightType, joinType,
+        typeFactory, fieldNameList, systemFieldList);
   }
 
-  /**
-   * Returns the type the row which results when two relations are joined.
-   *
-   * <p>The resulting row type consists of
-   * the system fields (if any), followed by
-   * the fields of the left type, followed by
-   * the fields of the right type. The field name list, if present, overrides
-   * the original names of the fields.
-   *
-   * @param typeFactory     Type factory
-   * @param leftType        Type of left input to join
-   * @param rightType       Type of right input to join
-   * @param fieldNameList   If not null, overrides the original names of the
-   *                        fields
-   * @param systemFieldList List of system fields that will be prefixed to
-   *                        output row type; typically empty but must not be
-   *                        null
-   * @return type of row which results when two relations are joined
-   */
+  @Deprecated // to be removed before 2.0
   public static RelDataType createJoinType(
       RelDataTypeFactory typeFactory,
       RelDataType leftType,
       RelDataType rightType,
       List<String> fieldNameList,
       List<RelDataTypeField> systemFieldList) {
-    assert (fieldNameList == null)
-        || (fieldNameList.size()
-        == (systemFieldList.size()
-        + leftType.getFieldCount()
-        + rightType.getFieldCount()));
-    List<String> nameList = new ArrayList<>();
-    final List<RelDataType> typeList = new ArrayList<>();
-
-    // use a hashset to keep track of the field names; this is needed
-    // to ensure that the contains() call to check for name uniqueness
-    // runs in constant time; otherwise, if the number of fields is large,
-    // doing a contains() on a list can be expensive
-    final HashSet<String> uniqueNameList = new HashSet<>();
-    addFields(systemFieldList, typeList, nameList, uniqueNameList);
-    addFields(leftType.getFieldList(), typeList, nameList, uniqueNameList);
-    if (rightType != null) {
-      addFields(
-          rightType.getFieldList(), typeList, nameList, uniqueNameList);
-    }
-    if (fieldNameList != null) {
-      assert fieldNameList.size() == nameList.size();
-      nameList = fieldNameList;
-    }
-    return typeFactory.createStructType(typeList, nameList);
-  }
-
-  private static void addFields(
-      List<RelDataTypeField> fieldList,
-      List<RelDataType> typeList,
-      List<String> nameList,
-      HashSet<String> uniqueNameList) {
-    for (RelDataTypeField field : fieldList) {
-      String name = field.getName();
-
-      // Ensure that name is unique from all previous field names
-      if (uniqueNameList.contains(name)) {
-        String nameBase = name;
-        for (int j = 0;; j++) {
-          name = nameBase + j;
-          if (!uniqueNameList.contains(name)) {
-            break;
-          }
-        }
-      }
-      nameList.add(name);
-      uniqueNameList.add(name);
-      typeList.add(field.getType());
-    }
+    return SqlValidatorUtil.createJoinType(typeFactory, leftType, rightType,
+        fieldNameList, systemFieldList);
   }
 
   @Override public final Join copy(RelTraitSet traitSet, List<RelNode> inputs) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java b/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
index 571e8f6..433ee5d 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
@@ -27,6 +27,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Util;
 
@@ -114,7 +115,7 @@ public class SemiJoin extends EquiJoin {
    * input only.
    */
   @Override public RelDataType deriveRowType() {
-    return deriveJoinRowType(
+    return SqlValidatorUtil.deriveJoinRowType(
         left.getRowType(),
         null,
         JoinRelType.INNER,

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
index 61f2fd7..6e9044e 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
@@ -80,7 +80,7 @@ public final class LogicalProject extends Project {
     this(cluster, cluster.traitSetOf(RelCollations.EMPTY),
         input, projects,
         RexUtil.createStructType(cluster.getTypeFactory(), projects,
-            fieldNames));
+            fieldNames, null));
     Util.discard(flags);
   }
 
@@ -97,14 +97,9 @@ public final class LogicalProject extends Project {
   public static LogicalProject create(final RelNode input,
       final List<? extends RexNode> projects, List<String> fieldNames) {
     final RelOptCluster cluster = input.getCluster();
-    final List<String> fieldNames2 =
-        fieldNames == null
-            ? null
-            : SqlValidatorUtil.uniquify(fieldNames,
-                SqlValidatorUtil.F_SUGGESTER);
     final RelDataType rowType =
         RexUtil.createStructType(cluster.getTypeFactory(), projects,
-            fieldNames2);
+            fieldNames, SqlValidatorUtil.F_SUGGESTER);
     return create(input, projects, rowType);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java b/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java
index f20fa89..45e824e 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java
@@ -595,10 +595,7 @@ public abstract class CalcRelSplitter {
     }
     if (outputRowType == null) {
       outputRowType =
-          RexUtil.createStructType(
-              typeFactory,
-              projectRefs,
-              fieldNames);
+          RexUtil.createStructType(typeFactory, projectRefs, fieldNames, null);
     }
     final RexProgram program =
         new RexProgram(

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
index c40c56a..39b4ef6 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
@@ -36,6 +36,7 @@ import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.Pair;
@@ -180,7 +181,7 @@ public class JoinProjectTransposeRule extends RelOptRule {
     // into the bottom RexProgram.  Note that the join type is an inner
     // join because the inputs haven't actually been joined yet.
     RelDataType joinChildrenRowType =
-        Join.deriveJoinRowType(
+        SqlValidatorUtil.deriveJoinRowType(
             leftJoinChild.getRowType(),
             rightJoinChild.getRowType(),
             JoinRelType.INNER,

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinProjectTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinProjectTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinProjectTransposeRule.java
index 6e945e5..8ce58ae 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinProjectTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinProjectTransposeRule.java
@@ -19,7 +19,6 @@ package org.apache.calcite.rel.rules;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.SemiJoin;
@@ -32,6 +31,7 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableIntList;
@@ -125,7 +125,7 @@ public class SemiJoinProjectTransposeRule extends RelOptRule {
     // for the bottom RexProgram, the input is a concatenation of the
     // child of the project and the RHS of the semijoin
     RelDataType bottomInputRowType =
-        Join.deriveJoinRowType(
+        SqlValidatorUtil.deriveJoinRowType(
             project.getInput().getRowType(),
             rightChild.getRowType(),
             JoinRelType.INNER,
@@ -156,7 +156,7 @@ public class SemiJoinProjectTransposeRule extends RelOptRule {
     // input rowtype into the top program is the concatenation of the
     // project and the RHS of the semijoin
     RelDataType topInputRowType =
-        Join.deriveJoinRowType(
+        SqlValidatorUtil.deriveJoinRowType(
             project.getRowType(),
             rightChild.getRowType(),
             JoinRelType.INNER,

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
index ab9b5bb..7c3e88c 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
@@ -290,7 +290,7 @@ public interface RelDataTypeFactory {
   /**
    * Callback which provides enough information to create fields.
    */
-  public interface FieldInfo {
+  interface FieldInfo {
     /**
      * Returns the number of fields.
      *
@@ -319,9 +319,9 @@ public interface RelDataTypeFactory {
    * Implementation of {@link FieldInfo} that provides a fluid API to build
    * a list of fields.
    */
-  public static class FieldInfoBuilder implements FieldInfo {
-    private final List<String> names = new ArrayList<String>();
-    private final List<RelDataType> types = new ArrayList<RelDataType>();
+  class FieldInfoBuilder implements FieldInfo {
+    private final List<String> names = new ArrayList<>();
+    private final List<RelDataType> types = new ArrayList<>();
 
     private final RelDataTypeFactory typeFactory;
 
@@ -422,7 +422,8 @@ public interface RelDataTypeFactory {
      * Makes sure that field names are unique.
      */
     public FieldInfoBuilder uniquify() {
-      final List<String> uniqueNames = SqlValidatorUtil.uniquify(names);
+      final List<String> uniqueNames = SqlValidatorUtil.uniquify(names,
+          typeFactory.getTypeSystem().isSchemaCaseSensitive());
       if (uniqueNames != names) {
         names.clear();
         names.addAll(uniqueNames);

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
index 34938f6..ff9d81f 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
@@ -73,6 +73,10 @@ public interface RelDataTypeSystem {
    * argument type.
    */
   RelDataType deriveSumType(RelDataTypeFactory typeFactory, RelDataType argumentType);
+
+  /** Whether two record types are considered distinct if their field names
+   * are the same but in different cases. */
+  boolean isSchemaCaseSensitive();
 }
 
 // End RelDataTypeSystem.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
index b760829..c34c2e6 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
@@ -45,7 +45,7 @@ public abstract class RelDataTypeSystemImpl implements RelDataTypeSystem {
     }
   }
 
-  public int getDefaultPrecision(SqlTypeName typeName) {
+  @Override public int getDefaultPrecision(SqlTypeName typeName) {
     //Following BasicSqlType precision as the default
     switch (typeName) {
     case CHAR:
@@ -86,7 +86,7 @@ public abstract class RelDataTypeSystemImpl implements RelDataTypeSystem {
     }
   }
 
-  public int getMaxPrecision(SqlTypeName typeName) {
+  @Override public int getMaxPrecision(SqlTypeName typeName) {
     switch (typeName) {
     case DECIMAL:
       return getMaxNumericPrecision();
@@ -107,15 +107,15 @@ public abstract class RelDataTypeSystemImpl implements RelDataTypeSystem {
     }
   }
 
-  public int getMaxNumericScale() {
+  @Override public int getMaxNumericScale() {
     return 19;
   }
 
-  public int getMaxNumericPrecision() {
+  @Override public int getMaxNumericPrecision() {
     return 19;
   }
 
-  public String getLiteral(SqlTypeName typeName, boolean isPrefix) {
+  @Override public String getLiteral(SqlTypeName typeName, boolean isPrefix) {
     switch(typeName) {
     case VARBINARY:
     case VARCHAR:
@@ -140,7 +140,7 @@ public abstract class RelDataTypeSystemImpl implements RelDataTypeSystem {
     }
   }
 
-  public boolean isCaseSensitive(SqlTypeName typeName) {
+  @Override public boolean isCaseSensitive(SqlTypeName typeName) {
     switch(typeName) {
     case CHAR:
     case VARCHAR:
@@ -150,11 +150,11 @@ public abstract class RelDataTypeSystemImpl implements RelDataTypeSystem {
     }
   }
 
-  public boolean isAutoincrement(SqlTypeName typeName) {
+  @Override public boolean isAutoincrement(SqlTypeName typeName) {
     return false;
   }
 
-  public int getNumTypeRadix(SqlTypeName typeName) {
+  @Override public int getNumTypeRadix(SqlTypeName typeName) {
     if (typeName.getFamily() == SqlTypeFamily.NUMERIC
       && getDefaultPrecision(typeName) != -1) {
       return 10;
@@ -162,10 +162,15 @@ public abstract class RelDataTypeSystemImpl implements RelDataTypeSystem {
     return 0;
   }
 
-  public RelDataType deriveSumType(
+  @Override public RelDataType deriveSumType(
       RelDataTypeFactory typeFactory, RelDataType argumentType) {
     return argumentType;
   }
+
+  public boolean isSchemaCaseSensitive() {
+    return true;
+  }
+
 }
 
 // End RelDataTypeSystemImpl.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
index 5f7d3ef..14128ec 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
@@ -423,10 +423,8 @@ public class RexProgramBuilder {
   }
 
   private RelDataType computeOutputRowType() {
-    return RexUtil.createStructType(
-        rexBuilder.typeFactory,
-        projectRefList,
-        projectNameList);
+    return RexUtil.createStructType(rexBuilder.typeFactory, projectRefList,
+        projectNameList, null);
   }
 
   private void generateMissingNames() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/rex/RexUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index d8221c6..e59b018 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -35,6 +35,7 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.ControlFlowException;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
@@ -653,7 +654,7 @@ public class RexUtil {
   public static RelDataType createStructType(
       RelDataTypeFactory typeFactory,
       final List<RexNode> exprs) {
-    return createStructType(typeFactory, exprs, null);
+    return createStructType(typeFactory, exprs, null, null);
   }
 
   /**
@@ -667,12 +668,19 @@ public class RexUtil {
    * @param typeFactory Type factory
    * @param exprs       Expressions
    * @param names       Field names, may be null, or elements may be null
+   * @param suggester   Generates alternative names if {@code names} is not
+   *                    null and its elements are not unique
    * @return Record type
    */
   public static RelDataType createStructType(
       RelDataTypeFactory typeFactory,
       final List<? extends RexNode> exprs,
-      final List<String> names) {
+      List<String> names,
+      SqlValidatorUtil.Suggester suggester) {
+    if (names != null && suggester != null) {
+      names = SqlValidatorUtil.uniquify(names, suggester,
+          typeFactory.getTypeSystem().isSchemaCaseSensitive());
+    }
     final RelDataTypeFactory.FieldInfoBuilder builder =
         typeFactory.builder();
     for (int i = 0; i < exprs.size(); i++) {
@@ -685,6 +693,14 @@ public class RexUtil {
     return builder.build();
   }
 
+  @Deprecated // to be removed before 2.0
+  public static RelDataType createStructType(
+      RelDataTypeFactory typeFactory,
+      final List<? extends RexNode> exprs,
+      List<String> names) {
+    return createStructType(typeFactory, exprs, names, null);
+  }
+
   /**
    * Returns whether the type of an array of expressions is compatible with a
    * struct type.

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/sql/SqlCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCall.java b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
index 620f1a8..542420c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
@@ -145,8 +145,9 @@ public abstract class SqlCall extends SqlNode {
     SqlCall that = (SqlCall) node;
 
     // Compare operators by name, not identity, because they may not
-    // have been resolved yet.
-    if (!this.getOperator().getName().equals(that.getOperator().getName())) {
+    // have been resolved yet. Use case insensitive comparison since
+    // this may be a case insensitive system.
+    if (!this.getOperator().getName().equalsIgnoreCase(that.getOperator().getName())) {
       return litmus.fail("{} != {}", this, node);
     }
     return equalDeep(this.getOperandList(), that.getOperandList(), litmus);

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
index 4a1c16b..b157c71 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
@@ -92,6 +92,8 @@ public interface SqlValidatorCatalogReader {
 
   RelDataType createTypeFromProjection(RelDataType type,
       List<String> columnNameList);
+
+  boolean isCaseSensitive();
 }
 
 // End SqlValidatorCatalogReader.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/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 10f4398..3aacce7 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
@@ -113,6 +113,7 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeSet;
 
 import static org.apache.calcite.sql.SqlUtil.stripAs;
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -314,7 +315,9 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
           select,
           unknownType,
           list,
-          new LinkedHashSet<String>(),
+          catalogReader.isCaseSensitive()
+          ? new LinkedHashSet<String>()
+              : new TreeSet<String>(String.CASE_INSENSITIVE_ORDER),
           types,
           includeSystemVars);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/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 ec67bc9..249f678 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
@@ -21,6 +21,7 @@ import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.plan.RelOptSchemaWithSampling;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.prepare.Prepare;
+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;
@@ -50,10 +51,12 @@ import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeSet;
 
 /**
  * Utility methods related to validation.
@@ -187,54 +190,85 @@ public class SqlValidatorUtil {
   }
 
   /**
+   * Factory method for {@link SqlValidator}.
+   */
+  public static SqlValidatorWithHints newValidator(
+      SqlOperatorTable opTab,
+      SqlValidatorCatalogReader catalogReader,
+      RelDataTypeFactory typeFactory,
+      SqlConformance conformance) {
+    return new SqlValidatorImpl(opTab, catalogReader, typeFactory,
+        conformance);
+  }
+
+  /**
+   * Factory method for {@link SqlValidator}, with default conformance.
+   */
+  @Deprecated // to be removed before 2.0
+  public static SqlValidatorWithHints newValidator(
+      SqlOperatorTable opTab,
+      SqlValidatorCatalogReader catalogReader,
+      RelDataTypeFactory typeFactory) {
+    return newValidator(opTab, catalogReader, typeFactory,
+        SqlConformance.DEFAULT);
+  }
+
+  /**
    * Makes a name distinct from other names which have already been used, adds
    * it to the list, and returns it.
    *
    * @param name      Suggested name, may not be unique
-   * @param nameList  Collection of names already used
+   * @param usedNames  Collection of names already used
    * @param suggester Base for name when input name is null
    * @return Unique name
    */
-  public static String uniquify(
-      String name,
-      Set<String> nameList,
+  public static String uniquify(String name, Set<String> usedNames,
       Suggester suggester) {
     if (name != null) {
-      if (nameList.add(name)) {
+      if (usedNames.add(name)) {
         return name;
       }
     }
     final String originalName = name;
     for (int j = 0;; j++) {
-      name = suggester.apply(originalName, j, nameList.size());
-      if (nameList.add(name)) {
+      name = suggester.apply(originalName, j, usedNames.size());
+      if (usedNames.add(name)) {
         return name;
       }
     }
   }
 
   /**
-   * Factory method for {@link SqlValidator}.
+   * Makes sure that the names in a list are unique.
+   *
+   * <p>Does not modify the input list. Returns the input list if the strings
+   * are unique, otherwise allocates a new list. Deprecated in favor of caseSensitive
+   * aware version.
+   *
+   * @param nameList List of strings
+   * @return List of unique strings
    */
-  public static SqlValidatorWithHints newValidator(
-      SqlOperatorTable opTab,
-      SqlValidatorCatalogReader catalogReader,
-      RelDataTypeFactory typeFactory,
-      SqlConformance conformance) {
-    return new SqlValidatorImpl(opTab, catalogReader, typeFactory,
-        conformance);
+  @Deprecated // to be removed before 2.0
+  public static List<String> uniquify(List<String> nameList) {
+    return uniquify(nameList, EXPR_SUGGESTER, true);
   }
 
+
   /**
-   * Factory method for {@link SqlValidator}, with default conformance.
+   * Makes sure that the names in a list are unique.
+   *
+   * <p>Does not modify the input list. Returns the input list if the strings
+   * are unique, otherwise allocates a new list.
+   *
+   * @deprecated Use {@link #uniquify(List, Suggester, boolean)}
+   *
+   * @param nameList List of strings
+   * @param suggester How to generate new names if duplicate names are found
+   * @return List of unique strings
    */
   @Deprecated // to be removed before 2.0
-  public static SqlValidatorWithHints newValidator(
-      SqlOperatorTable opTab,
-      SqlValidatorCatalogReader catalogReader,
-      RelDataTypeFactory typeFactory) {
-    return newValidator(opTab, catalogReader, typeFactory,
-        SqlConformance.DEFAULT);
+  public static List<String> uniquify(List<String> nameList, Suggester suggester) {
+    return uniquify(nameList, suggester, true);
   }
 
   /**
@@ -244,26 +278,162 @@ public class SqlValidatorUtil {
    * are unique, otherwise allocates a new list.
    *
    * @param nameList List of strings
+   * @param caseSensitive Whether upper and lower case names are considered
+   *     distinct
    * @return List of unique strings
    */
-  public static List<String> uniquify(List<String> nameList) {
-    return uniquify(nameList, EXPR_SUGGESTER);
+  public static List<String> uniquify(List<String> nameList,
+      boolean caseSensitive) {
+    return uniquify(nameList, EXPR_SUGGESTER, caseSensitive);
   }
 
+  /**
+   * Makes sure that the names in a list are unique.
+   *
+   * <p>Does not modify the input list. Returns the input list if the strings
+   * are unique, otherwise allocates a new list.
+   *
+   * @param nameList List of strings
+   * @param suggester How to generate new names if duplicate names are found
+   * @param caseSensitive Whether upper and lower case names are considered
+   *     distinct
+   * @return List of unique strings
+   */
   public static List<String> uniquify(
       List<String> nameList,
-      Suggester suggester) {
-    final Set<String> used = new LinkedHashSet<>();
+      Suggester suggester,
+      boolean caseSensitive) {
+    final Set<String> used = caseSensitive
+        ? new LinkedHashSet<String>()
+        : new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
     int changeCount = 0;
+    final List<String> newNameList = new ArrayList<>();
     for (String name : nameList) {
       String uniqueName = uniquify(name, used, suggester);
       if (!uniqueName.equals(name)) {
         ++changeCount;
       }
+      newNameList.add(uniqueName);
     }
     return changeCount == 0
         ? nameList
-        : new ArrayList<>(used);
+        : newNameList;
+  }
+
+  /**
+   * Derives the type of a join relational expression.
+   *
+   * @param leftType        Row type of left input to join
+   * @param rightType       Row type of right input to join
+   * @param joinType        Type of join
+   * @param typeFactory     Type factory
+   * @param fieldNameList   List of names of fields; if null, field names are
+   *                        inherited and made unique
+   * @param systemFieldList List of system fields that will be prefixed to
+   *                        output row type; typically empty but must not be
+   *                        null
+   * @return join type
+   */
+  public static RelDataType deriveJoinRowType(
+      RelDataType leftType,
+      RelDataType rightType,
+      JoinRelType joinType,
+      RelDataTypeFactory typeFactory,
+      List<String> fieldNameList,
+      List<RelDataTypeField> systemFieldList) {
+    assert systemFieldList != null;
+    switch (joinType) {
+    case LEFT:
+      rightType = typeFactory.createTypeWithNullability(rightType, true);
+      break;
+    case RIGHT:
+      leftType = typeFactory.createTypeWithNullability(leftType, true);
+      break;
+    case FULL:
+      leftType = typeFactory.createTypeWithNullability(leftType, true);
+      rightType = typeFactory.createTypeWithNullability(rightType, true);
+      break;
+    default:
+      break;
+    }
+    return createJoinType(typeFactory, leftType, rightType, fieldNameList,
+        systemFieldList);
+  }
+
+  /**
+   * Returns the type the row which results when two relations are joined.
+   *
+   * <p>The resulting row type consists of
+   * the system fields (if any), followed by
+   * the fields of the left type, followed by
+   * the fields of the right type. The field name list, if present, overrides
+   * the original names of the fields.
+   *
+   * @param typeFactory     Type factory
+   * @param leftType        Type of left input to join
+   * @param rightType       Type of right input to join
+   * @param fieldNameList   If not null, overrides the original names of the
+   *                        fields
+   * @param systemFieldList List of system fields that will be prefixed to
+   *                        output row type; typically empty but must not be
+   *                        null
+   * @return type of row which results when two relations are joined
+   */
+  public static RelDataType createJoinType(
+      RelDataTypeFactory typeFactory,
+      RelDataType leftType,
+      RelDataType rightType,
+      List<String> fieldNameList,
+      List<RelDataTypeField> systemFieldList) {
+    assert (fieldNameList == null)
+        || (fieldNameList.size()
+        == (systemFieldList.size()
+        + leftType.getFieldCount()
+        + rightType.getFieldCount()));
+    List<String> nameList = new ArrayList<>();
+    final List<RelDataType> typeList = new ArrayList<>();
+
+    // Use a set to keep track of the field names; this is needed
+    // to ensure that the contains() call to check for name uniqueness
+    // runs in constant time; otherwise, if the number of fields is large,
+    // doing a contains() on a list can be expensive.
+    final Set<String> uniqueNameList =
+        typeFactory.getTypeSystem().isSchemaCaseSensitive()
+            ? new HashSet<String>()
+            : new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+    addFields(systemFieldList, typeList, nameList, uniqueNameList);
+    addFields(leftType.getFieldList(), typeList, nameList, uniqueNameList);
+    if (rightType != null) {
+      addFields(
+          rightType.getFieldList(), typeList, nameList, uniqueNameList);
+    }
+    if (fieldNameList != null) {
+      assert fieldNameList.size() == nameList.size();
+      nameList = fieldNameList;
+    }
+    return typeFactory.createStructType(typeList, nameList);
+  }
+
+  private static void addFields(List<RelDataTypeField> fieldList,
+      List<RelDataType> typeList, List<String> nameList,
+      Set<String> uniqueNames) {
+    for (RelDataTypeField field : fieldList) {
+      String name = field.getName();
+
+      // Ensure that name is unique from all previous field names
+      if (uniqueNames.contains(name)) {
+        String nameBase = name;
+        for (int j = 0;; j++) {
+          name = nameBase + j;
+          if (!uniqueNames.contains(name)) {
+            break;
+          }
+        }
+      }
+      nameList.add(name);
+      uniqueNames.add(name);
+      typeList.add(field.getType());
+    }
   }
 
   /**
@@ -685,7 +855,7 @@ public class SqlValidatorUtil {
 
   /** Suggests candidates for unique names, given the number of attempts so far
    * and the number of expressions in the project list. */
-  interface Suggester {
+  public interface Suggester {
     String apply(String original, int attempt, int size);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/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 6b5e34a..8b13e26 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -454,7 +454,8 @@ public class SqlToRelConverter {
     final RelDataType validatedRowType =
         validator.getTypeFactory().createStructType(
             Pair.right(validatedFields),
-            SqlValidatorUtil.uniquify(Pair.left(validatedFields)));
+            SqlValidatorUtil.uniquify(Pair.left(validatedFields),
+                catalogReader.isCaseSensitive()));
 
     final List<RelDataTypeField> convertedFields =
         result.getRowType().getFieldList().subList(0, validatedFields.size());
@@ -3520,7 +3521,7 @@ public class SqlToRelConverter {
       fieldNames.add(deriveAlias(expr, aliases, i));
     }
 
-    fieldNames = SqlValidatorUtil.uniquify(fieldNames);
+    fieldNames = SqlValidatorUtil.uniquify(fieldNames, catalogReader.isCaseSensitive());
 
     bb.setRoot(
         RelOptUtil.createProject(bb.root, exprs, fieldNames),

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/test/java/org/apache/calcite/sql/validate/LexCaseSensitiveTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/validate/LexCaseSensitiveTest.java b/core/src/test/java/org/apache/calcite/sql/validate/LexCaseSensitiveTest.java
new file mode 100644
index 0000000..589a466
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/sql/validate/LexCaseSensitiveTest.java
@@ -0,0 +1,203 @@
+/*
+ * 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.sql.validate;
+
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableProject;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParser.Config;
+import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
+import org.apache.calcite.tools.Program;
+import org.apache.calcite.tools.Programs;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.hamcrest.CoreMatchers.anyOf;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Testing {@link SqlValidator} and {@link Lex}.
+ */
+public class LexCaseSensitiveTest {
+
+  private static Planner getPlanner(List<RelTraitDef> traitDefs,
+      SqlParser.Config parserConfig, Program... programs) {
+    final SchemaPlus rootSchema = Frameworks.createRootSchema(true);
+    final FrameworkConfig config = Frameworks.newConfigBuilder()
+        .parserConfig(parserConfig)
+        .defaultSchema(CalciteAssert.addSchema(rootSchema, CalciteAssert.SchemaSpec.HR))
+        .traitDefs(traitDefs)
+        .programs(programs)
+        .build();
+    return Frameworks.getPlanner(config);
+  }
+
+  private static void runProjectQueryWithLex(Lex lex, String sql)
+      throws SqlParseException, ValidationException, RelConversionException {
+    Config javaLex = SqlParser.configBuilder().setLex(lex).build();
+    Planner planner = getPlanner(null, javaLex, Programs.ofRules(Programs.RULE_SET));
+    SqlNode parse = planner.parse(sql);
+    SqlNode validate = planner.validate(parse);
+    RelNode convert = planner.rel(validate).rel;
+    RelTraitSet traitSet =
+        planner.getEmptyTraitSet().replace(EnumerableConvention.INSTANCE);
+    RelNode transform = planner.transform(0, traitSet, convert);
+    assertThat(transform, instanceOf(EnumerableProject.class));
+    List<String> fieldNames = transform.getRowType().getFieldNames();
+    assertThat(fieldNames.size(), is(2));
+    if (lex.caseSensitive) {
+      assertThat(fieldNames.get(0), is("EMPID"));
+      assertThat(fieldNames.get(1), is("empid"));
+    } else {
+      assertThat(fieldNames.get(0) + "-" + fieldNames.get(1),
+          anyOf(is("EMPID-empid0"), is("EMPID0-empid")));
+    }
+  }
+
+  @Test public void testCalciteCaseOracle()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select \"empid\" as EMPID, \"empid\" from\n"
+        + " (select \"empid\" from \"emps\" order by \"emps\".\"deptno\")";
+    runProjectQueryWithLex(Lex.ORACLE, sql);
+  }
+
+  @Test(expected = ValidationException.class)
+  public void testCalciteCaseOracleException()
+      throws SqlParseException, ValidationException, RelConversionException {
+    // Oracle is case sensitive, so EMPID should not be found.
+    String sql = "select EMPID, \"empid\" from\n"
+        + " (select \"empid\" from \"emps\" order by \"emps\".\"deptno\")";
+    runProjectQueryWithLex(Lex.ORACLE, sql);
+  }
+
+  @Test public void testCalciteCaseMySql()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select empid as EMPID, empid from (\n"
+        + "  select empid from emps order by `EMPS`.DEPTNO)";
+    runProjectQueryWithLex(Lex.MYSQL, sql);
+  }
+
+  @Test public void testCalciteCaseMySqlNoException()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select EMPID, empid from\n"
+        + " (select empid from emps order by emps.deptno)";
+    runProjectQueryWithLex(Lex.MYSQL, sql);
+  }
+
+  @Test public void testCalciteCaseMySqlAnsi()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select empid as EMPID, empid from (\n"
+        + "  select empid from emps order by EMPS.DEPTNO)";
+    runProjectQueryWithLex(Lex.MYSQL_ANSI, sql);
+  }
+
+  @Test public void testCalciteCaseMySqlAnsiNoException()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select EMPID, empid from\n"
+        + " (select empid from emps order by emps.deptno)";
+    runProjectQueryWithLex(Lex.MYSQL_ANSI, sql);
+  }
+
+  @Test public void testCalciteCaseSqlServer()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select empid as EMPID, empid from (\n"
+        + "  select empid from emps order by EMPS.DEPTNO)";
+    runProjectQueryWithLex(Lex.SQL_SERVER, sql);
+  }
+
+  @Test public void testCalciteCaseSqlServerNoException()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select EMPID, empid from\n"
+        + " (select empid from emps order by emps.deptno)";
+    runProjectQueryWithLex(Lex.SQL_SERVER, sql);
+  }
+
+  @Test public void testCalciteCaseJava()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select empid as EMPID, empid from (\n"
+        + "  select empid from emps order by emps.deptno)";
+    runProjectQueryWithLex(Lex.JAVA, sql);
+  }
+
+  @Test(expected = ValidationException.class)
+  public void testCalciteCaseJavaException()
+      throws SqlParseException, ValidationException, RelConversionException {
+    // JAVA is case sensitive, so EMPID should not be found.
+    String sql = "select EMPID, empid from\n"
+        + " (select empid from emps order by emps.deptno)";
+    runProjectQueryWithLex(Lex.JAVA, sql);
+  }
+
+  @Test public void testCalciteCaseJoinOracle()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select t.\"empid\" as EMPID, s.\"empid\" from\n"
+        + "(select * from \"emps\" where \"emps\".\"deptno\" > 100) t join\n"
+        + "(select * from \"emps\" where \"emps\".\"deptno\" < 200) s\n"
+        + "on t.\"empid\" = s.\"empid\"";
+    runProjectQueryWithLex(Lex.ORACLE, sql);
+  }
+
+  @Test public void testCalciteCaseJoinMySql()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select t.empid as EMPID, s.empid from\n"
+        + "(select * from emps where emps.deptno > 100) t join\n"
+        + "(select * from emps where emps.deptno < 200) s on t.empid = s.empid";
+    runProjectQueryWithLex(Lex.MYSQL, sql);
+  }
+
+  @Test public void testCalciteCaseJoinMySqlAnsi()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select t.empid as EMPID, s.empid from\n"
+        + "(select * from emps where emps.deptno > 100) t join\n"
+        + "(select * from emps where emps.deptno < 200) s on t.empid = s.empid";
+    runProjectQueryWithLex(Lex.MYSQL_ANSI, sql);
+  }
+
+  @Test public void testCalciteCaseJoinSqlServer()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select t.empid as EMPID, s.empid from\n"
+        + "(select * from emps where emps.deptno > 100) t join\n"
+        + "(select * from emps where emps.deptno < 200) s on t.empid = s.empid";
+    runProjectQueryWithLex(Lex.SQL_SERVER, sql);
+  }
+
+  @Test public void testCalciteCaseJoinJava()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select t.empid as EMPID, s.empid from\n"
+        + "(select * from emps where emps.deptno > 100) t join\n"
+        + "(select * from emps where emps.deptno < 200) s on t.empid = s.empid";
+    runProjectQueryWithLex(Lex.JAVA, sql);
+  }
+}
+
+// End LexCaseSensitiveTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/test/java/org/apache/calcite/sql/validate/SqlValidatorUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/validate/SqlValidatorUtilTest.java b/core/src/test/java/org/apache/calcite/sql/validate/SqlValidatorUtilTest.java
new file mode 100644
index 0000000..87de5f5
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/sql/validate/SqlValidatorUtilTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.sql.validate;
+
+import com.google.common.collect.Lists;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.hamcrest.CoreMatchers.anyOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.CoreMatchers.sameInstance;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link SqlValidatorUtil}.
+ */
+public class SqlValidatorUtilTest {
+
+  private static void checkChangedFieldList(
+      List<String> nameList, List<String> resultList, boolean caseSensitive) {
+    // Check that the new names are appended with "0" in order they appear in
+    // original nameList. This is assuming that we only have one "collision".
+    int i = 0;
+    for (String name : nameList) {
+      String newName = resultList.get(i);
+      assertThat(newName, anyOf(is(name), is(name + "0")));
+      i++;
+    }
+
+    // Make sure each name is unique
+    List<String> copyResultList  = new ArrayList<>(resultList.size());
+    for (String result : resultList) {
+      copyResultList.add(result.toLowerCase());
+    }
+
+    for (String result : resultList) {
+      assertThat(copyResultList.contains(result.toLowerCase()), is(true));
+      copyResultList.remove(result.toLowerCase());
+      if (!caseSensitive) {
+        assertThat(copyResultList.contains(result.toLowerCase()), is(false));
+      }
+    }
+    assertThat(copyResultList.size(), is(0));
+  }
+
+  @Test public void testUniquifyCaseSensitive() {
+    List<String> nameList = Lists.newArrayList("col1", "COL1", "col_ABC", "col_abC");
+    List<String> resultList = SqlValidatorUtil.uniquify(
+        nameList, SqlValidatorUtil.EXPR_SUGGESTER, true);
+    assertThat(nameList, sameInstance(resultList));
+  }
+
+  @Test public void testUniquifyNotCaseSensitive() {
+    List<String> nameList = Lists.newArrayList("col1", "COL1", "col_ABC", "col_abC");
+    List<String> resultList = SqlValidatorUtil.uniquify(
+        nameList, SqlValidatorUtil.EXPR_SUGGESTER, false);
+    assertThat(resultList, not(nameList));
+    checkChangedFieldList(nameList, resultList, false);
+  }
+
+  @Test public void testUniquifyOrderingCaseSensitive() {
+    List<String> nameList = Lists.newArrayList("k68s", "def", "col1", "COL1", "abc", "123");
+    List<String> resultList = SqlValidatorUtil.uniquify(
+        nameList, SqlValidatorUtil.EXPR_SUGGESTER, true);
+    assertThat(nameList, sameInstance(resultList));
+  }
+
+  @Test public void testUniquifyOrderingRepeatedCaseSensitive() {
+    List<String> nameList = Lists.newArrayList("k68s", "def", "col1", "COL1", "def", "123");
+    List<String> resultList = SqlValidatorUtil.uniquify(
+        nameList, SqlValidatorUtil.EXPR_SUGGESTER, true);
+    assertThat(nameList, not(resultList));
+    checkChangedFieldList(nameList, resultList, true);
+  }
+
+  @Test public void testUniquifyOrderingNotCaseSensitive() {
+    List<String> nameList = Lists.newArrayList("k68s", "def", "col1", "COL1", "abc", "123");
+    List<String> resultList = SqlValidatorUtil.uniquify(
+        nameList, SqlValidatorUtil.EXPR_SUGGESTER, false);
+    assertThat(resultList, not(nameList));
+    checkChangedFieldList(nameList, resultList, false);
+  }
+
+  @Test public void testUniquifyOrderingRepeatedNotCaseSensitive() {
+    List<String> nameList = Lists.newArrayList("k68s", "def", "col1", "COL1", "def", "123");
+    List<String> resultList = SqlValidatorUtil.uniquify(
+        nameList, SqlValidatorUtil.EXPR_SUGGESTER, false);
+    assertThat(resultList, not(nameList));
+    checkChangedFieldList(nameList, resultList, false);
+  }
+
+}
+
+// End SqlValidatorUtilTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
index 9bab603..c1320d2 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -39,6 +39,8 @@ import org.apache.calcite.sql.test.SqlOperatorTest;
 import org.apache.calcite.sql.test.SqlPrettyWriterTest;
 import org.apache.calcite.sql.test.SqlTypeNameTest;
 import org.apache.calcite.sql.type.SqlTypeFactoryTest;
+import org.apache.calcite.sql.validate.LexCaseSensitiveTest;
+import org.apache.calcite.sql.validate.SqlValidatorUtilTest;
 import org.apache.calcite.test.enumerable.EnumerableCorrelateTest;
 import org.apache.calcite.tools.FrameworksTest;
 import org.apache.calcite.tools.PlannerTest;
@@ -99,6 +101,7 @@ import org.junit.runners.Suite;
     RelOptPlanReaderTest.class,
     RexBuilderTest.class,
     SqlTypeFactoryTest.class,
+    SqlValidatorUtilTest.class,
 
     // medium tests (above 0.1s)
     SqlParserTest.class,
@@ -121,6 +124,7 @@ import org.junit.runners.Suite;
     FrameworksTest.class,
     EnumerableCorrelateTest.class,
     LookupOperatorOverloadsTest.class,
+    LexCaseSensitiveTest.class,
 
     // slow tests (above 1s)
     UdfTest.class,

http://git-wip-us.apache.org/repos/asf/calcite/blob/acd27fde/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 c58f19f..0ccedc2 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -135,6 +135,10 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     }
   }
 
+  @Override public boolean isCaseSensitive() {
+    return caseSensitive;
+  }
+
   /**
    * Initializes this catalog reader.
    */