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:29 UTC

[1/5] calcite git commit: [CALCITE-991] Create separate SqlFunctionCategory values for table functions and macros (Julien Le Dem, Minji Kim)

Repository: calcite
Updated Branches:
  refs/heads/master a2d7165f3 -> 94051eaed


[CALCITE-991] Create separate SqlFunctionCategory values for table functions and macros (Julien Le Dem, Minji Kim)

Add a simple test for CalciteCatalogReader (Minji Kim).

Fix up (Julian Hyde).

Close apache/calcite#168


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

Branch: refs/heads/master
Commit: 0599cdde009d6e08f0cb973f914b526f9518dc47
Parents: a2d7165
Author: Julien Le Dem <ju...@dremio.com>
Authored: Sun Mar 13 22:06:30 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jun 14 18:02:58 2016 -0700

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       |   4 +-
 .../calcite/prepare/CalciteCatalogReader.java   |  23 ++-
 .../apache/calcite/rel/externalize/RelJson.java |   6 +-
 .../org/apache/calcite/sql/SqlFunction.java     |   7 +-
 .../apache/calcite/sql/SqlFunctionCategory.java |  66 ++++++-
 .../java/org/apache/calcite/sql/SqlUtil.java    |   3 +-
 .../sql/parser/SqlAbstractParserImpl.java       |   2 +-
 .../calcite/sql/util/ListSqlOperatorTable.java  |  19 +-
 .../apache/calcite/sql/validate/AggFinder.java  |  24 +--
 .../sql/validate/SqlUserDefinedFunction.java    |  15 +-
 .../validate/SqlUserDefinedTableFunction.java   |   3 +-
 .../sql/validate/SqlUserDefinedTableMacro.java  |   2 +-
 .../sql2rel/RelStructuredTypeFlattener.java     |  11 +-
 .../prepare/LookupOperatorOverloadsTest.java    | 174 +++++++++++++++++++
 .../org/apache/calcite/test/CalciteSuite.java   |   2 +
 15 files changed, 308 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/core/src/main/codegen/templates/Parser.jj
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index bad32b0..d7415cc 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -1928,13 +1928,13 @@ void ColumnType(List<SqlNode> list) :
 SqlNode TableFunctionCall(SqlParserPos pos) :
 {
     SqlNode call;
-    SqlFunctionCategory funcType = SqlFunctionCategory.USER_DEFINED_FUNCTION;
+    SqlFunctionCategory funcType = SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION;
 }
 {
     [
         <SPECIFIC>
         {
-            funcType = SqlFunctionCategory.USER_DEFINED_SPECIFIC_FUNCTION;
+            funcType = SqlFunctionCategory.USER_DEFINED_TABLE_SPECIFIC_FUNCTION;
         }
     ]
     {

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/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 68b297e..40299e9 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -53,6 +53,7 @@ import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
 import com.google.common.collect.Collections2;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
@@ -232,7 +233,27 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
     if (syntax != SqlSyntax.FUNCTION) {
       return;
     }
-    final Collection<Function> functions = getFunctionsFrom(opName.names);
+
+    final Predicate<Function> predicate;
+    if (category == null) {
+      predicate = Predicates.alwaysTrue();
+    } else if (category.isTableFunction()) {
+      predicate = new Predicate<Function>() {
+        public boolean apply(Function function) {
+          return function instanceof TableMacro
+              || function instanceof TableFunction;
+        }
+      };
+    } else {
+      predicate = new Predicate<Function>() {
+        public boolean apply(Function function) {
+          return !(function instanceof TableMacro
+              || function instanceof TableFunction);
+        }
+      };
+    }
+    final Collection<Function> functions =
+        Collections2.filter(getFunctionsFrom(opName.names), predicate);
     if (functions.isEmpty()) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java b/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
index b7ef54a..1b24811 100644
--- a/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
@@ -339,11 +339,7 @@ public class RelJson {
           map.put("type", toJson(node.getType()));
         }
         if (call.getOperator() instanceof SqlFunction) {
-          switch (((SqlFunction) call.getOperator()).getFunctionType()) {
-          case USER_DEFINED_CONSTRUCTOR:
-          case USER_DEFINED_FUNCTION:
-          case USER_DEFINED_PROCEDURE:
-          case USER_DEFINED_SPECIFIC_FUNCTION:
+          if (((SqlFunction) call.getOperator()).getFunctionType().isUserDefined()) {
             map.put("class", call.getOperator().getClass().getName());
           }
         }

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
index 3bdc91c..7f9b0f3 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
@@ -26,9 +26,11 @@ import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
+import javax.annotation.Nonnull;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
@@ -121,10 +123,9 @@ public class SqlFunction extends SqlOperator {
         operandTypeChecker);
 
     this.sqlIdentifier = sqlIdentifier;
-    this.category = category;
+    this.category = Preconditions.checkNotNull(category);
     this.paramTypes =
         paramTypes == null ? null : ImmutableList.copyOf(paramTypes);
-    assert category != null;
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -174,7 +175,7 @@ public class SqlFunction extends SqlOperator {
   /**
    * @return function category
    */
-  public SqlFunctionCategory getFunctionType() {
+  @Nonnull public SqlFunctionCategory getFunctionType() {
     return this.category;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/core/src/main/java/org/apache/calcite/sql/SqlFunctionCategory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlFunctionCategory.java b/core/src/main/java/org/apache/calcite/sql/SqlFunctionCategory.java
index 8c836da..02cd381 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlFunctionCategory.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlFunctionCategory.java
@@ -18,24 +18,72 @@ package org.apache.calcite.sql;
 
 import org.apache.calcite.util.Util;
 
+import static org.apache.calcite.sql.SqlFunctionCategory.Property.FUNCTION;
+import static org.apache.calcite.sql.SqlFunctionCategory.Property.SPECIFIC;
+import static org.apache.calcite.sql.SqlFunctionCategory.Property.TABLE_FUNCTION;
+import static org.apache.calcite.sql.SqlFunctionCategory.Property.USER_DEFINED;
+
+import java.util.Arrays;
+import java.util.EnumSet;
+
 /**
  * Enumeration of the categories of
  * SQL-invoked routines.
  */
 public enum SqlFunctionCategory {
-  STRING("STRING", "String function"),
-  NUMERIC("NUMERIC", "Numeric function"),
-  TIMEDATE("TIMEDATE", "Time and date function"),
-  SYSTEM("SYSTEM", "System function"),
-  USER_DEFINED_FUNCTION("UDF", "User-defined function"),
-  USER_DEFINED_PROCEDURE("UDP", "User-defined procedure"),
-  USER_DEFINED_CONSTRUCTOR("UDC", "User-defined constructor"),
+  STRING("STRING", "String function", FUNCTION),
+  NUMERIC("NUMERIC", "Numeric function", FUNCTION),
+  TIMEDATE("TIMEDATE", "Time and date function", FUNCTION),
+  SYSTEM("SYSTEM", "System function", FUNCTION),
+  USER_DEFINED_FUNCTION("UDF", "User-defined function", USER_DEFINED,
+      FUNCTION),
+  USER_DEFINED_PROCEDURE("UDP", "User-defined procedure", USER_DEFINED),
+  USER_DEFINED_CONSTRUCTOR("UDC", "User-defined constructor", USER_DEFINED),
   USER_DEFINED_SPECIFIC_FUNCTION("UDF_SPECIFIC",
-      "User-defined function with SPECIFIC name");
+      "User-defined function with SPECIFIC name", USER_DEFINED, SPECIFIC,
+      FUNCTION),
+  USER_DEFINED_TABLE_FUNCTION("TABLE_UDF", "User-defined table function",
+      USER_DEFINED, TABLE_FUNCTION),
+  USER_DEFINED_TABLE_SPECIFIC_FUNCTION("TABLE_UDF_SPECIFIC",
+      "User-defined table function with SPECIFIC name", USER_DEFINED,
+      TABLE_FUNCTION, SPECIFIC);
 
-  SqlFunctionCategory(String abbrev, String description) {
+  private final EnumSet<Property> properties;
+
+  SqlFunctionCategory(String abbrev, String description,
+      Property... properties) {
     Util.discard(abbrev);
     Util.discard(description);
+    this.properties = EnumSet.copyOf(Arrays.asList(properties));
+  }
+
+  public boolean isUserDefined() {
+    return properties.contains(USER_DEFINED);
+  }
+
+  public boolean isTableFunction() {
+    return properties.contains(TABLE_FUNCTION);
+  }
+
+  public boolean isFunction() {
+    return properties.contains(FUNCTION);
+  }
+
+  public boolean isSpecific() {
+    return properties.contains(SPECIFIC);
+  }
+
+  public boolean isUserDefinedNotSpecificFunction() {
+    return isUserDefined()
+        && (isFunction() || isTableFunction())
+        && !isSpecific();
+  }
+
+  /**
+   * Property of a SqlFunctionCategory.
+   */
+  enum Property {
+    USER_DEFINED, TABLE_FUNCTION, SPECIFIC, FUNCTION
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
index a620f2a..ca8fb27 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -263,8 +263,7 @@ public abstract class SqlUtil {
     if (operator instanceof SqlFunction) {
       SqlFunction function = (SqlFunction) operator;
 
-      switch (function.getFunctionType()) {
-      case USER_DEFINED_SPECIFIC_FUNCTION:
+      if (function.getFunctionType().isSpecific()) {
         writer.keyword("SPECIFIC");
       }
       SqlIdentifier id = function.getSqlIdentifier();

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
index bef5da7..2e0008d 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
@@ -363,7 +363,7 @@ public abstract class SqlAbstractParserImpl {
     /// name when regenerating SQL).
     if (funName.isSimple()) {
       final List<SqlOperator> list = Lists.newArrayList();
-      opTab.lookupOperatorOverloads(funName, null, SqlSyntax.FUNCTION, list);
+      opTab.lookupOperatorOverloads(funName, funcType, SqlSyntax.FUNCTION, list);
       if (list.size() == 1) {
         fun = list.get(0);
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/core/src/main/java/org/apache/calcite/sql/util/ListSqlOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/util/ListSqlOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/util/ListSqlOperatorTable.java
index b018177..fe0aea0 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/ListSqlOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/ListSqlOperatorTable.java
@@ -63,20 +63,23 @@ public class ListSqlOperatorTable implements SqlOperatorTable {
           || !operator.isName(opName.getSimple())) {
         continue;
       }
-      SqlFunctionCategory functionCategory;
-      if (operator instanceof SqlFunction) {
-        functionCategory = ((SqlFunction) operator).getFunctionType();
-      } else {
-        functionCategory = SqlFunctionCategory.SYSTEM;
-      }
-      if (category != functionCategory
-          && category != SqlFunctionCategory.USER_DEFINED_FUNCTION) {
+      if (category != null
+          && category != category(operator)
+          && !category.isUserDefinedNotSpecificFunction()) {
         continue;
       }
       operatorList.add(operator);
     }
   }
 
+  protected static SqlFunctionCategory category(SqlOperator operator) {
+    if (operator instanceof SqlFunction) {
+      return ((SqlFunction) operator).getFunctionType();
+    } else {
+      return SqlFunctionCategory.SYSTEM;
+    }
+  }
+
   public List<SqlOperator> getOperatorList() {
     return operatorList;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java b/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
index a363fca..29226b0 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
@@ -18,7 +18,6 @@ package org.apache.calcite.sql.validate;
 
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlFunction;
-import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperator;
@@ -110,17 +109,18 @@ class AggFinder extends SqlBasicVisitor<Void> {
       }
     }
     // User-defined function may not be resolved yet.
-    if (operator instanceof SqlFunction
-        && ((SqlFunction) operator).getFunctionType()
-        == SqlFunctionCategory.USER_DEFINED_FUNCTION) {
-      final List<SqlOperator> list = Lists.newArrayList();
-      opTab.lookupOperatorOverloads(((SqlFunction) operator).getSqlIdentifier(),
-          SqlFunctionCategory.USER_DEFINED_FUNCTION, SqlSyntax.FUNCTION, list);
-      for (SqlOperator sqlOperator : list) {
-        if (sqlOperator.isAggregator()) {
-          // If nested aggregates disallowed or found aggregate at invalid level
-          if (aggregate) {
-            throw new Util.FoundOne(call);
+    if (operator instanceof SqlFunction) {
+      final SqlFunction sqlFunction = (SqlFunction) operator;
+      if (sqlFunction.getFunctionType().isUserDefinedNotSpecificFunction()) {
+        final List<SqlOperator> list = Lists.newArrayList();
+        opTab.lookupOperatorOverloads(sqlFunction.getSqlIdentifier(),
+            sqlFunction.getFunctionType(), SqlSyntax.FUNCTION, list);
+        for (SqlOperator sqlOperator : list) {
+          if (sqlOperator.isAggregator()) {
+            // If nested aggregates disallowed or found aggregate at invalid level
+            if (aggregate) {
+              throw new Util.FoundOne(call);
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java
index 163687e..cc3ed3f 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java
@@ -41,15 +41,28 @@ import java.util.List;
 public class SqlUserDefinedFunction extends SqlFunction {
   public final Function function;
 
+  /** Creates a {@link SqlUserDefinedFunction}. */
   public SqlUserDefinedFunction(SqlIdentifier opName,
       SqlReturnTypeInference returnTypeInference,
       SqlOperandTypeInference operandTypeInference,
       SqlOperandTypeChecker operandTypeChecker,
       List<RelDataType> paramTypes,
       Function function) {
+    this(opName, returnTypeInference, operandTypeInference, operandTypeChecker,
+        paramTypes, function, SqlFunctionCategory.USER_DEFINED_FUNCTION);
+  }
+
+  /** Constructor used internally and by derived classes. */
+  protected SqlUserDefinedFunction(SqlIdentifier opName,
+      SqlReturnTypeInference returnTypeInference,
+      SqlOperandTypeInference operandTypeInference,
+      SqlOperandTypeChecker operandTypeChecker,
+      List<RelDataType> paramTypes,
+      Function function,
+      SqlFunctionCategory category) {
     super(Util.last(opName.names), opName, SqlKind.OTHER_FUNCTION,
         returnTypeInference, operandTypeInference, operandTypeChecker,
-        paramTypes, SqlFunctionCategory.USER_DEFINED_FUNCTION);
+        paramTypes, category);
     this.function = function;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
index 338a1ed..1e6f300 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
@@ -19,6 +19,7 @@ package org.apache.calcite.sql.validate;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.TableFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
@@ -42,7 +43,7 @@ public class SqlUserDefinedTableFunction extends SqlUserDefinedFunction {
       List<RelDataType> paramTypes,
       TableFunction function) {
     super(opName, returnTypeInference, operandTypeInference, operandTypeChecker,
-        paramTypes, function);
+        paramTypes, function, SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
index e20eed0..9ef6cc2 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
@@ -70,7 +70,7 @@ public class SqlUserDefinedTableMacro extends SqlFunction {
     super(Util.last(opName.names), opName, SqlKind.OTHER_FUNCTION,
         returnTypeInference, operandTypeInference, operandTypeChecker,
         Preconditions.checkNotNull(paramTypes),
-        SqlFunctionCategory.USER_DEFINED_FUNCTION);
+        SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION);
     this.tableMacro = tableMacro;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java b/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
index e7278d6..82b1f4e 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
@@ -702,8 +702,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
             RelStructuredTypeFlattener.class,
             RelNode.class);
 
-    // implement RelVisitor
-    public void visit(RelNode p, int ordinal, RelNode parent) {
+    @Override public void visit(RelNode p, int ordinal, RelNode parent) {
       // rewrite children first
       super.visit(p, ordinal, parent);
 
@@ -720,13 +719,11 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
           // for leaves, it's usually safe to assume that
           // no transformation is required
           rewriteGeneric(p);
+        } else {
+          throw new AssertionError("no '" + visitMethodName
+              + "' method found for class " + p.getClass().getName());
         }
       }
-      if (!found) {
-        throw Util.newInternal(
-            "no '" + visitMethodName + "' method found for class "
-            + p.getClass().getName());
-      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/core/src/test/java/org/apache/calcite/prepare/LookupOperatorOverloadsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/prepare/LookupOperatorOverloadsTest.java b/core/src/test/java/org/apache/calcite/prepare/LookupOperatorOverloadsTest.java
new file mode 100644
index 0000000..e760cd0
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/prepare/LookupOperatorOverloadsTest.java
@@ -0,0 +1,174 @@
+/*
+ * 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.prepare;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.TableFunction;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.schema.impl.TableFunctionImpl;
+import org.apache.calcite.server.CalciteServerStatement;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.validate.SqlUserDefinedTableFunction;
+import org.apache.calcite.util.Smalls;
+
+import com.google.common.collect.Lists;
+
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.calcite.sql.SqlFunctionCategory.USER_DEFINED_CONSTRUCTOR;
+import static org.apache.calcite.sql.SqlFunctionCategory.USER_DEFINED_FUNCTION;
+import static org.apache.calcite.sql.SqlFunctionCategory.USER_DEFINED_PROCEDURE;
+import static org.apache.calcite.sql.SqlFunctionCategory.USER_DEFINED_SPECIFIC_FUNCTION;
+import static org.apache.calcite.sql.SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION;
+import static org.apache.calcite.sql.SqlFunctionCategory.USER_DEFINED_TABLE_SPECIFIC_FUNCTION;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for lookupOperatorOverloads() in {@link CalciteCatalogReader}.
+ */
+public class LookupOperatorOverloadsTest {
+
+  private void checkFunctionType(int size, String name,
+      List<SqlOperator> operatorList) {
+    assertThat(size, is(operatorList.size()));
+
+    for (SqlOperator op : operatorList) {
+      assertThat(op, instanceOf(SqlUserDefinedTableFunction.class));
+      assertThat(name, is(op.getName()));
+    }
+  }
+
+  private static void check(List<SqlFunctionCategory> actuals,
+      SqlFunctionCategory... expecteds) {
+    assertThat(actuals, is(Arrays.asList(expecteds)));
+  }
+
+  @Test public void testIsUserDefined() throws SQLException {
+    List<SqlFunctionCategory> cats = new ArrayList<>();
+    for (SqlFunctionCategory c : SqlFunctionCategory.values()) {
+      if (c.isUserDefined()) {
+        cats.add(c);
+      }
+    }
+    check(cats, USER_DEFINED_FUNCTION, USER_DEFINED_PROCEDURE,
+        USER_DEFINED_CONSTRUCTOR, USER_DEFINED_SPECIFIC_FUNCTION,
+        USER_DEFINED_TABLE_FUNCTION, USER_DEFINED_TABLE_SPECIFIC_FUNCTION);
+  }
+
+  @Test public void testIsTableFunction() throws SQLException {
+    List<SqlFunctionCategory> cats = new ArrayList<>();
+    for (SqlFunctionCategory c : SqlFunctionCategory.values()) {
+      if (c.isTableFunction()) {
+        cats.add(c);
+      }
+    }
+    check(cats, USER_DEFINED_TABLE_FUNCTION,
+        USER_DEFINED_TABLE_SPECIFIC_FUNCTION);
+  }
+
+  @Test public void testIsSpecific() throws SQLException {
+    List<SqlFunctionCategory> cats = new ArrayList<>();
+    for (SqlFunctionCategory c : SqlFunctionCategory.values()) {
+      if (c.isSpecific()) {
+        cats.add(c);
+      }
+    }
+    check(cats, USER_DEFINED_SPECIFIC_FUNCTION,
+        USER_DEFINED_TABLE_SPECIFIC_FUNCTION);
+  }
+
+  @Test public void testIsUserDefinedNotSpecificFunction() throws SQLException {
+    List<SqlFunctionCategory> cats = new ArrayList<>();
+    for (SqlFunctionCategory sqlFunctionCategory : SqlFunctionCategory.values()) {
+      if (sqlFunctionCategory.isUserDefinedNotSpecificFunction()) {
+        cats.add(sqlFunctionCategory);
+      }
+    }
+    check(cats, USER_DEFINED_FUNCTION, USER_DEFINED_TABLE_FUNCTION);
+  }
+
+  @Test public void test() throws SQLException {
+    final String schemaName = "MySchema";
+    final String funcName = "MyFUNC";
+    final String anotherName = "AnotherFunc";
+
+    try (Connection connection = DriverManager.getConnection("jdbc:calcite:")) {
+      CalciteConnection calciteConnection =
+          connection.unwrap(CalciteConnection.class);
+      SchemaPlus rootSchema = calciteConnection.getRootSchema();
+      SchemaPlus schema = rootSchema.add(schemaName, new AbstractSchema());
+      final TableFunction table = TableFunctionImpl.create(Smalls.MAZE_METHOD);
+      schema.add(funcName, table);
+      schema.add(anotherName, table);
+      final TableFunction table2 =
+          TableFunctionImpl.create(Smalls.MAZE3_METHOD);
+      schema.add(funcName, table2);
+
+      final CalciteServerStatement statement =
+          connection.createStatement().unwrap(CalciteServerStatement.class);
+      final CalcitePrepare.Context prepareContext =
+          statement.createPrepareContext();
+      final JavaTypeFactory typeFactory = prepareContext.getTypeFactory();
+      CalciteCatalogReader reader =
+          new CalciteCatalogReader(prepareContext.getRootSchema(), false, null,
+              typeFactory);
+
+      final List<SqlOperator> operatorList = new ArrayList<>();
+      SqlIdentifier myFuncIdentifier =
+          new SqlIdentifier(Lists.newArrayList(schemaName, funcName), null,
+              SqlParserPos.ZERO, null);
+      reader.lookupOperatorOverloads(myFuncIdentifier,
+          SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION, SqlSyntax.FUNCTION,
+          operatorList);
+      checkFunctionType(2, funcName, operatorList);
+
+      operatorList.clear();
+      reader.lookupOperatorOverloads(myFuncIdentifier,
+          SqlFunctionCategory.USER_DEFINED_FUNCTION, SqlSyntax.FUNCTION,
+          operatorList);
+      checkFunctionType(0, null, operatorList);
+
+      operatorList.clear();
+      SqlIdentifier anotherFuncIdentifier =
+          new SqlIdentifier(Lists.newArrayList(schemaName, anotherName), null,
+              SqlParserPos.ZERO, null);
+      reader.lookupOperatorOverloads(anotherFuncIdentifier,
+          SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION, SqlSyntax.FUNCTION,
+          operatorList);
+      checkFunctionType(1, anotherName, operatorList);
+    }
+  }
+}
+
+// End LookupOperatorOverloadsTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/0599cdde/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 3dfb659..9bab603 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -24,6 +24,7 @@ import org.apache.calcite.plan.RelWriterTest;
 import org.apache.calcite.plan.volcano.TraitPropagationTest;
 import org.apache.calcite.plan.volcano.VolcanoPlannerTest;
 import org.apache.calcite.plan.volcano.VolcanoPlannerTraitTest;
+import org.apache.calcite.prepare.LookupOperatorOverloadsTest;
 import org.apache.calcite.rel.RelCollationTest;
 import org.apache.calcite.rel.rel2sql.RelToSqlConverterTest;
 import org.apache.calcite.rex.RexBuilderTest;
@@ -119,6 +120,7 @@ import org.junit.runners.Suite;
     ChunkListTest.class,
     FrameworksTest.class,
     EnumerableCorrelateTest.class,
+    LookupOperatorOverloadsTest.class,
 
     // slow tests (above 1s)
     UdfTest.class,


[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)

Posted by jh...@apache.org.
[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.
    */


[2/5] calcite git commit: [CALCITE-1283] leastRestrictiveSqlType() should account for nullability of each type and null types (Minji Kim)

Posted by jh...@apache.org.
[CALCITE-1283] leastRestrictiveSqlType() should account for nullability of each type and null types (Minji Kim)

Close apache/calcite#246


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

Branch: refs/heads/master
Commit: 0404349e84b6286aa16add5ce9ae6728240ffdbc
Parents: acd27fd
Author: Minji Kim <mi...@dremio.com>
Authored: Fri Jun 10 13:28:59 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jun 14 18:02:59 2016 -0700

----------------------------------------------------------------------
 .../calcite/sql/type/SqlTypeFactoryImpl.java    |  6 +-
 .../calcite/sql/type/SqlTypeFactoryTest.java    | 58 ++++++++++++++------
 2 files changed, 44 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/0404349e/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
index c7636cc..57378cc 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
@@ -226,11 +226,9 @@ public class SqlTypeFactoryImpl extends RelDataTypeFactoryImpl {
       if (typeName == null) {
         return null;
       }
-
       if (typeName == SqlTypeName.ANY) {
         anyCount++;
       }
-
       if (type.isNullable()) {
         ++nullableCount;
       }
@@ -242,10 +240,10 @@ public class SqlTypeFactoryImpl extends RelDataTypeFactoryImpl {
       }
     }
 
-
     //  if any of the inputs are ANY, the output is ANY
     if (anyCount > 0) {
-      return createTypeWithNullability(createSqlType(SqlTypeName.ANY), nullCount > 0);
+      return createTypeWithNullability(createSqlType(SqlTypeName.ANY),
+          nullCount > 0 || nullableCount > 0);
     }
 
     for (int i = 0; i < types.size(); ++i) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/0404349e/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFactoryTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFactoryTest.java b/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFactoryTest.java
index 5748b14..4ca75fa 100644
--- a/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFactoryTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/type/SqlTypeFactoryTest.java
@@ -23,33 +23,59 @@ import com.google.common.collect.Lists;
 
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
 
 /**
- * Test for {@link SqlTypeFactoryImpl}
+ * Test for {@link SqlTypeFactoryImpl}.
  */
 public class SqlTypeFactoryTest {
 
-  @Test
-  public void testLeastRestrictiveWithAny() {
-    SqlTypeFactoryImpl typeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
-    final RelDataType sqlBigInt = typeFactory.createSqlType(SqlTypeName.BIGINT);
-    final RelDataType sqlAny = typeFactory.createSqlType(SqlTypeName.ANY);
+  @Test public void testLeastRestrictiveWithAny() {
+    Fixture f = new Fixture();
+    RelDataType leastRestrictive =
+        f.typeFactory.leastRestrictive(Lists.newArrayList(f.sqlBigInt, f.sqlAny));
+    assertThat(leastRestrictive.getSqlTypeName(), is(SqlTypeName.ANY));
+  }
 
+  @Test public void testLeastRestrictiveWithNumbers() {
+    Fixture f = new Fixture();
     RelDataType leastRestrictive =
-            typeFactory.leastRestrictive(Lists.newArrayList(sqlBigInt, sqlAny));
-    assertEquals(leastRestrictive.getSqlTypeName(), SqlTypeName.ANY);
+        f.typeFactory.leastRestrictive(Lists.newArrayList(f.sqlBigInt, f.sqlInt));
+    assertThat(leastRestrictive.getSqlTypeName(), is(SqlTypeName.BIGINT));
   }
 
-  @Test
-  public void testLeastRestrictiveWithNumbers() {
-    SqlTypeFactoryImpl typeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
-    final RelDataType sqlBigInt = typeFactory.createSqlType(SqlTypeName.BIGINT);
-    final RelDataType sqlInt = typeFactory.createSqlType(SqlTypeName.INTEGER);
+  @Test public void testLeastRestrictiveWithNullability() {
+    Fixture f = new Fixture();
+    RelDataType leastRestrictive =
+        f.typeFactory.leastRestrictive(Lists.newArrayList(f.sqlVarcharNullable, f.sqlAny));
+    assertThat(leastRestrictive.getSqlTypeName(), is(SqlTypeName.ANY));
+    assertThat(leastRestrictive.isNullable(), is(true));
+  }
 
+  @Test public void testLeastRestrictiveWithNull() {
+    Fixture f = new Fixture();
     RelDataType leastRestrictive =
-            typeFactory.leastRestrictive(Lists.newArrayList(sqlBigInt, sqlInt));
-    assertEquals(leastRestrictive.getSqlTypeName(), SqlTypeName.BIGINT);
+        f.typeFactory.leastRestrictive(Lists.newArrayList(f.sqlNull, f.sqlNull));
+    assertThat(leastRestrictive.getSqlTypeName(), is(SqlTypeName.NULL));
+    assertThat(leastRestrictive.isNullable(), is(true));
   }
+
+  /** Sets up data needed by a test. */
+  private static class Fixture {
+    SqlTypeFactoryImpl typeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+    final RelDataType sqlBigInt = typeFactory.createTypeWithNullability(
+        typeFactory.createSqlType(SqlTypeName.BIGINT), false);
+    final RelDataType sqlInt = typeFactory.createTypeWithNullability(
+        typeFactory.createSqlType(SqlTypeName.INTEGER), false);
+    final RelDataType sqlVarcharNullable = typeFactory.createTypeWithNullability(
+        typeFactory.createSqlType(SqlTypeName.VARCHAR), true);
+    final RelDataType sqlNull = typeFactory.createTypeWithNullability(
+        typeFactory.createSqlType(SqlTypeName.NULL), false);
+    final RelDataType sqlAny = typeFactory.createTypeWithNullability(
+        typeFactory.createSqlType(SqlTypeName.ANY), false);
+  }
+
 }
+
 // End SqlTypeFactoryTest.java


[3/5] calcite git commit: [CALCITE-1278] CalciteSignature's ColumnMetaData info for DELETE should be same as INSERT

Posted by jh...@apache.org.
[CALCITE-1278] CalciteSignature's ColumnMetaData info for DELETE should be same as INSERT


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

Branch: refs/heads/master
Commit: 76616379a8018679230930c4f8d16c51ecb35c59
Parents: 0404349
Author: maryannxue <ma...@gmail.com>
Authored: Tue Jun 14 15:07:20 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jun 14 18:02:59 2016 -0700

----------------------------------------------------------------------
 .../enumerable/EnumerableTableModify.java       | 30 ++++++++++--
 .../org/apache/calcite/plan/RelOptUtil.java     |  1 +
 .../calcite/prepare/CalcitePrepareImpl.java     | 22 +++++----
 .../org/apache/calcite/util/BuiltInMethod.java  |  1 +
 .../calcite/test/JdbcFrontLinqBackTest.java     | 23 +++++++++
 .../java/org/apache/calcite/test/JdbcTest.java  | 50 +++++++++++++++++++-
 .../calcite/linq4j/DefaultEnumerable.java       |  4 ++
 .../calcite/linq4j/EnumerableDefaults.java      | 10 ++++
 .../calcite/linq4j/ExtendedEnumerable.java      |  7 ++-
 9 files changed, 131 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/76616379/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java
index c40678e..adcd417 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java
@@ -31,6 +31,7 @@ import org.apache.calcite.rel.core.TableModify;
 import org.apache.calcite.schema.ModifiableTable;
 import org.apache.calcite.util.BuiltInMethod;
 
+import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -121,20 +122,39 @@ public class EnumerableTableModify extends TableModify
     } else {
       convertedChildExp = childExp;
     }
+    final Method method;
+    switch (getOperation()) {
+    case INSERT:
+      method = BuiltInMethod.INTO.method;
+      break;
+    case DELETE:
+      method = BuiltInMethod.REMOVE_ALL.method;
+      break;
+    default:
+      throw new AssertionError(getOperation());
+    }
     builder.add(
         Expressions.statement(
             Expressions.call(
-                convertedChildExp, "into", collectionParameter)));
+                convertedChildExp, method, collectionParameter)));
+    final Expression updatedCountParameter =
+        builder.append(
+            "updatedCount",
+            Expressions.call(collectionParameter, "size"),
+            false);
     builder.add(
         Expressions.return_(
             null,
             Expressions.call(
                 BuiltInMethod.SINGLETON_ENUMERABLE.method,
                 Expressions.convert_(
-                    Expressions.subtract(
-                        Expressions.call(
-                            collectionParameter, "size"),
-                        countParameter),
+                    Expressions.condition(
+                        Expressions.greaterThanOrEqual(
+                            updatedCountParameter, countParameter),
+                        Expressions.subtract(
+                            updatedCountParameter, countParameter),
+                        Expressions.subtract(
+                            countParameter, updatedCountParameter)),
                     long.class))));
     final PhysType physType =
         PhysTypeImpl.of(

http://git-wip-us.apache.org/repos/asf/calcite/blob/76616379/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 60e9cb7..d9d2a63 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -1730,6 +1730,7 @@ public abstract class RelOptUtil {
       RelDataTypeFactory typeFactory) {
     switch (kind) {
     case INSERT:
+    case DELETE:
       return typeFactory.createStructType(
           ImmutableList.of(
               Pair.of(AvaticaConnection.ROWCOUNT_COLUMN_NAME,

http://git-wip-us.apache.org/repos/asf/calcite/blob/76616379/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index bd53b7b..60a76bb 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -632,24 +632,27 @@ public class CalcitePrepareImpl implements CalcitePrepare {
   }
 
   /**
-   * Routine to figure out the StatementType and defaults to SELECT
-   * As CASE increases the default may change
-   * @param kind a SqlKind
-   * @return Meta.StatementType*/
+   * Deduces the broad type of statement.
+   * Currently returns SELECT for most statement types, but this may change.
+   *
+   * @param kind Kind of statement
+   */
   private Meta.StatementType getStatementType(SqlKind kind) {
     switch (kind) {
     case INSERT:
-      return Meta.StatementType.INSERT;
+    case DELETE:
+      return Meta.StatementType.IS_DML;
     default:
       return Meta.StatementType.SELECT;
     }
   }
 
   /**
-   * Routine to figure out the StatementType if call does not have sql
-   * defaults to SELECT
-   * @param preparedResult An objecet returned from prepareQueryable or prepareRel
-   * @return Meta.StatementType*/
+   * Deduces the broad type of statement for a prepare result.
+   * Currently returns SELECT for most statement types, but this may change.
+   *
+   * @param preparedResult Prepare result
+   */
   private Meta.StatementType getStatementType(Prepare.PreparedResult preparedResult) {
     if (preparedResult.isDml()) {
       return Meta.StatementType.IS_DML;
@@ -725,6 +728,7 @@ public class CalcitePrepareImpl implements CalcitePrepare {
           sqlNode, Object.class, validator, true);
       switch (sqlNode.getKind()) {
       case INSERT:
+      case DELETE:
       case EXPLAIN:
         // FIXME: getValidatedNodeType is wrong for DML
         x = RelOptUtil.createDmlRowType(sqlNode.getKind(), typeFactory);

http://git-wip-us.apache.org/repos/asf/calcite/blob/76616379/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index 6a17843..9850c9f 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -115,6 +115,7 @@ public enum BuiltInMethod {
   AS_QUERYABLE(Enumerable.class, "asQueryable"),
   ABSTRACT_ENUMERABLE_CTOR(AbstractEnumerable.class),
   INTO(ExtendedEnumerable.class, "into", Collection.class),
+  REMOVE_ALL(ExtendedEnumerable.class, "removeAll", Collection.class),
   SCHEMA_GET_SUB_SCHEMA(Schema.class, "getSubSchema", String.class),
   SCHEMA_GET_TABLE(Schema.class, "getTable", String.class),
   SCHEMA_PLUS_UNWRAP(SchemaPlus.class, "unwrap", Class.class),

http://git-wip-us.apache.org/repos/asf/calcite/blob/76616379/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java b/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
index 58c7783..8a2171a 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
@@ -274,6 +274,29 @@ public class JdbcFrontLinqBackTest {
             "empid=1; deptno=0; name=foo; salary=10.0; commission=null");
   }
 
+  @Test public void testDelete() {
+    final List<JdbcTest.Employee> employees = new ArrayList<>();
+    CalciteAssert.AssertThat with = mutable(employees);
+    with.query("select * from \"foo\".\"bar\"")
+        .returnsUnordered(
+            "empid=0; deptno=0; name=first; salary=0.0; commission=null");
+    with.query("insert into \"foo\".\"bar\" select * from \"hr\".\"emps\"")
+        .updates(4);
+    with.query("select count(*) as c from \"foo\".\"bar\"")
+        .returnsUnordered("C=5");
+    final String deleteSql = "delete from \"foo\".\"bar\" "
+        + "where \"deptno\" = 10";
+    with.query(deleteSql)
+        .updates(3);
+    final String sql = "select \"name\", count(*) as c\n"
+        + "from \"foo\".\"bar\"\n"
+        + "group by \"name\"";
+    with.query(sql)
+        .returnsUnordered(
+            "name=Eric; C=1",
+            "name=first; C=1");
+  }
+
   /**
    * Creates the post processor routine to be applied against a Connection.
    *

http://git-wip-us.apache.org/repos/asf/calcite/blob/76616379/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index e8aec78..fc2c396 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -128,6 +128,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
+import java.util.Objects;
 import java.util.Properties;
 import java.util.Set;
 import java.util.TimeZone;
@@ -6479,10 +6480,16 @@ public class JdbcTest {
       this.commission = commission;
     }
 
-    public String toString() {
+    @Override public String toString() {
       return "Employee [empid: " + empid + ", deptno: " + deptno
           + ", name: " + name + "]";
     }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof Employee
+          && empid == ((Employee) obj).empid;
+    }
   }
 
   public static class Department {
@@ -6501,10 +6508,16 @@ public class JdbcTest {
       this.location = location;
     }
 
-    public String toString() {
+    @Override public String toString() {
       return "Department [deptno: " + deptno + ", name: " + name
           + ", employees: " + employees + ", location: " + location + "]";
     }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof Department
+          && deptno == ((Department) obj).deptno;
+    }
   }
 
   public static class Location {
@@ -6519,6 +6532,13 @@ public class JdbcTest {
     @Override public String toString() {
       return "Location [x: " + x + ", y: " + y + "]";
     }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof Location
+          && x == ((Location) obj).x
+          && y == ((Location) obj).y;
+    }
   }
 
   public static class Dependent {
@@ -6533,6 +6553,13 @@ public class JdbcTest {
     @Override public String toString() {
       return "Dependent [empid: " + empid + ", name: " + name + "]";
     }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof Dependent
+          && empid == ((Dependent) obj).empid
+          && Objects.equals(name, ((Dependent) obj).name);
+    }
   }
 
   public static class FoodmartSchema {
@@ -6557,6 +6584,12 @@ public class JdbcTest {
       this.EMPNO = EMPNO;
       this.DEPTNO = DEPTNO;
     }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof LingualEmp
+          && EMPNO == ((LingualEmp) obj).EMPNO;
+    }
   }
 
   public static class FoodmartJdbcSchema extends JdbcSchema {
@@ -6574,6 +6607,12 @@ public class JdbcTest {
     public Customer(int customer_id) {
       this.customer_id = customer_id;
     }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof Customer
+          && customer_id == ((Customer) obj).customer_id;
+    }
   }
 
   public static class SalesFact {
@@ -6584,6 +6623,13 @@ public class JdbcTest {
       this.cust_id = cust_id;
       this.prod_id = prod_id;
     }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof SalesFact
+          && cust_id == ((SalesFact) obj).cust_id
+          && prod_id == ((SalesFact) obj).prod_id;
+    }
   }
 
   //CHECKSTYLE: ON

http://git-wip-us.apache.org/repos/asf/calcite/blob/76616379/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultEnumerable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultEnumerable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultEnumerable.java
index e3c8916..146c175 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultEnumerable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultEnumerable.java
@@ -343,6 +343,10 @@ public abstract class DefaultEnumerable<T> implements OrderedEnumerable<T> {
     return EnumerableDefaults.into(getThis(), sink);
   }
 
+  public <C extends Collection<? super T>> C removeAll(C sink) {
+    return EnumerableDefaults.remove(getThis(), sink);
+  }
+
   public <TInner, TKey, TResult> Enumerable<TResult> join(
       Enumerable<TInner> inner, Function1<T, TKey> outerKeySelector,
       Function1<TInner, TKey> innerKeySelector,

http://git-wip-us.apache.org/repos/asf/calcite/blob/76616379/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java b/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
index dc3ef03..e715cdb 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
@@ -2902,6 +2902,7 @@ public abstract class EnumerableDefaults {
             source, (Class) Object.class, null, null);
   }
 
+  /** Default implementation of {@link ExtendedEnumerable#into(Collection)}. */
   public static <T, C extends Collection<? super T>> C into(
       Enumerable<T> source, C sink) {
     try (Enumerator<T> enumerator = source.enumerator()) {
@@ -2913,6 +2914,15 @@ public abstract class EnumerableDefaults {
     return sink;
   }
 
+  /** Default implementation of {@link ExtendedEnumerable#removeAll(Collection)}. */
+  public static <T, C extends Collection<? super T>> C remove(
+      Enumerable<T> source, C sink) {
+    List<T> tempList = new ArrayList<>();
+    source.into(tempList);
+    sink.removeAll(tempList);
+    return sink;
+  }
+
   /** Enumerable that implements take-while. */
   static class TakeWhileEnumerator<TSource> implements Enumerator<TSource> {
     private final Enumerator<TSource> enumerator;

http://git-wip-us.apache.org/repos/asf/calcite/blob/76616379/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedEnumerable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedEnumerable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedEnumerable.java
index 316a289..626901e 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedEnumerable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedEnumerable.java
@@ -480,11 +480,16 @@ public interface ExtendedEnumerable<TSource> {
       EqualityComparer<TSource> comparer);
 
   /**
-   * Copies the contents of the sequence into a collection.
+   * Copies the contents of this sequence into a collection.
    */
   <C extends Collection<? super TSource>> C into(C sink);
 
   /**
+   * Removes the contents of this sequence from a collection.
+   */
+  <C extends Collection<? super TSource>> C removeAll(C sink);
+
+  /**
    * Correlates the elements of two sequences based on
    * matching keys. The default equality comparer is used to compare
    * keys.


[4/5] calcite git commit: [CALCITE-1289] RexUtil.simplifyCase() should account for nullability (Minji Kim)

Posted by jh...@apache.org.
[CALCITE-1289] RexUtil.simplifyCase() should account for nullability (Minji Kim)

Close apache/calcite#248


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

Branch: refs/heads/master
Commit: 94051eaed884bd07a03f69ab20c9e1496aac4862
Parents: 7661637
Author: Minji Kim <mi...@dremio.com>
Authored: Fri Jun 10 18:02:58 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jun 14 18:02:59 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/calcite/rex/RexUtil.java    |  10 +-
 .../org/apache/calcite/test/RexProgramTest.java | 166 +++++++++++++------
 2 files changed, 120 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/94051eae/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 e59b018..5f06102 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -1566,7 +1566,15 @@ public class RexUtil {
           notTerms.add(pair.e.getKey());
         }
       }
-      return composeDisjunction(rexBuilder, terms, false);
+      RexNode disjunction = composeDisjunction(rexBuilder, terms, false);
+
+      assert call.getType().getSqlTypeName() == disjunction.getType().getSqlTypeName();
+      if (call.getType().isNullable() == disjunction.getType().isNullable()) {
+        return disjunction;
+      } else if (call.getType().isNullable() && !disjunction.getType().isNullable()) {
+        return rexBuilder.ensureType(call.getType(), disjunction, false);
+      }
+      // if call is not nullable, but the disjunction is, we should not use the disjunction.
     }
     if (newOperands.equals(operands)) {
       return call;

http://git-wip-us.apache.org/repos/asf/calcite/blob/94051eae/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
index 9907959..e142c72 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -32,6 +32,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexProgramBuilder;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.ImmutableBitSet;
@@ -60,6 +61,10 @@ public class RexProgramTest {
   //~ Instance fields --------------------------------------------------------
   private JavaTypeFactory typeFactory;
   private RexBuilder rexBuilder;
+  private RexLiteral trueLiteral;
+  private RexLiteral falseLiteral;
+  private RexNode nullLiteral;
+  private RexNode unknownLiteral;
 
   //~ Methods ----------------------------------------------------------------
 
@@ -74,6 +79,10 @@ public class RexProgramTest {
   public void setUp() {
     typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
     rexBuilder = new RexBuilder(typeFactory);
+    trueLiteral = rexBuilder.makeLiteral(true);
+    falseLiteral = rexBuilder.makeLiteral(false);
+    nullLiteral = rexBuilder.makeNullLiteral(SqlTypeName.INTEGER);
+    unknownLiteral = rexBuilder.makeNullLiteral(SqlTypeName.BOOLEAN);
   }
 
   private void checkCnf(RexNode node, String expected) {
@@ -235,7 +244,8 @@ public class RexProgramTest {
         is("(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], expr#3=[1], "
             + "expr#4=[+($t0, $t3)], expr#5=[+($t2, $t4)], "
             + "expr#6=[+($t0, $t4)], expr#7=[5], expr#8=[>($t4, $t7)], "
-            + "expr#9=[NOT($t8)], a=[$t5], b=[$t6], $condition=[$t9])"));
+            + "expr#9=[CAST($t8):BOOLEAN], expr#10=[IS FALSE($t9)], "
+            + "a=[$t5], b=[$t6], $condition=[$t10])"));
   }
 
   /**
@@ -256,7 +266,8 @@ public class RexProgramTest {
         is("(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], expr#3=[1], "
             + "expr#4=[+($t0, $t3)], expr#5=[+($t2, $t4)], "
             + "expr#6=[+($t0, $t4)], expr#7=[5], expr#8=[>($t4, $t7)], "
-            + "expr#9=[NOT($t8)], a=[$t5], b=[$t6], $condition=[$t9])"));
+            + "expr#9=[CAST($t8):BOOLEAN], expr#10=[IS FALSE($t9)], "
+            + "a=[$t5], b=[$t6], $condition=[$t10])"));
   }
 
   /**
@@ -391,10 +402,7 @@ public class RexProgramTest {
       // $t8 = $t7 AND $t7
       t8 =
           builder.addExpr(
-              rexBuilder.makeCall(
-                  SqlStdOperatorTable.AND,
-                  t7,
-                  t7));
+              and(t7, t7));
       builder.addCondition(t8);
       builder.addCondition(t7);
       break;
@@ -403,12 +411,10 @@ public class RexProgramTest {
       // $t7 = 5
       t7 = builder.addExpr(c5);
       // $t8 = $t2 > $t7 (i.e. (x + 1) > 5)
-      t8 =
-          builder.addExpr(
-              rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t2, t7));
+      t8 = builder.addExpr(gt(t2, t7));
       // $t9 = true
       final RexLocalRef t9 =
-          builder.addExpr(rexBuilder.makeLiteral(true));
+          builder.addExpr(trueLiteral);
       // $t10 = $t1 is not null (i.e. y is not null)
       assert t1 != null;
       final RexLocalRef t10 =
@@ -416,19 +422,16 @@ public class RexProgramTest {
               rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, t1));
       // $t11 = false
       final RexLocalRef t11 =
-          builder.addExpr(rexBuilder.makeLiteral(false));
+          builder.addExpr(falseLiteral);
       // $t12 = unknown
       final RexLocalRef t12 =
-          builder.addExpr(rexBuilder.makeNullLiteral(SqlTypeName.BOOLEAN));
+          builder.addExpr(unknownLiteral);
       // $t13 = case when $t8 then $t9 when $t10 then $t11 else $t12 end
       final RexLocalRef t13 =
-          builder.addExpr(
-              rexBuilder.makeCall(SqlStdOperatorTable.CASE,
-                  t8, t9, t10, t11, t12));
+          builder.addExpr(case_(t8, t9, t10, t11, t12));
       // $t14 = not $t13 (i.e. not case ... end)
       final RexLocalRef t14 =
-          builder.addExpr(
-              rexBuilder.makeCall(SqlStdOperatorTable.NOT, t13));
+          builder.addExpr(not(t13));
       // don't add 't14 is true' - that is implicit
       if (variant == 3) {
         builder.addCondition(t14);
@@ -466,12 +469,6 @@ public class RexProgramTest {
     assertThat(strongIf(aRef, c13), is(false));
 
     // literals are strong iff they are always null
-    final RexLiteral trueLiteral = rexBuilder.makeLiteral(true);
-    final RexLiteral falseLiteral = rexBuilder.makeLiteral(false);
-    final RexNode nullLiteral = rexBuilder.makeNullLiteral(SqlTypeName.INTEGER);
-    final RexNode unknownLiteral =
-        rexBuilder.makeNullLiteral(SqlTypeName.BOOLEAN);
-
     assertThat(strongIf(trueLiteral, c), is(false));
     assertThat(strongIf(trueLiteral, c13), is(false));
     assertThat(strongIf(falseLiteral, c13), is(false));
@@ -480,15 +477,9 @@ public class RexProgramTest {
     assertThat(strongIf(unknownLiteral, c13), is(true));
 
     // AND is strong if one of its arguments is strong
-    final RexNode andUnknownTrue =
-        rexBuilder.makeCall(SqlStdOperatorTable.AND,
-            unknownLiteral, trueLiteral);
-    final RexNode andTrueUnknown =
-        rexBuilder.makeCall(SqlStdOperatorTable.AND,
-            trueLiteral, unknownLiteral);
-    final RexNode andFalseTrue =
-        rexBuilder.makeCall(SqlStdOperatorTable.AND,
-            falseLiteral, trueLiteral);
+    final RexNode andUnknownTrue = and(unknownLiteral, trueLiteral);
+    final RexNode andTrueUnknown = and(trueLiteral, unknownLiteral);
+    final RexNode andFalseTrue = and(falseLiteral, trueLiteral);
 
     assertThat(strongIf(andUnknownTrue, c), is(true));
     assertThat(strongIf(andTrueUnknown, c), is(true));
@@ -525,11 +516,6 @@ public class RexProgramTest {
         rexBuilder.makeExactLiteral(BigDecimal.valueOf(7));
     final RexNode hEqSeven = eq(hRef, sevenLiteral);
 
-    final RexLiteral trueLiteral = rexBuilder.makeLiteral(true);
-    final RexLiteral falseLiteral = rexBuilder.makeLiteral(false);
-    final RexNode unknownLiteral =
-        rexBuilder.makeNullLiteral(SqlTypeName.BOOLEAN);
-
     checkCnf(aRef, "?0.a");
     checkCnf(trueLiteral, "true");
     checkCnf(falseLiteral, "false");
@@ -711,11 +697,6 @@ public class RexProgramTest {
         rexBuilder.makeExactLiteral(BigDecimal.valueOf(7));
     final RexNode hEqSeven = eq(hRef, sevenLiteral);
 
-    final RexLiteral trueLiteral = rexBuilder.makeLiteral(true);
-    final RexLiteral falseLiteral = rexBuilder.makeLiteral(false);
-    final RexNode unknownLiteral =
-        rexBuilder.makeNullLiteral(SqlTypeName.BOOLEAN);
-
     // Most of the expressions in testCnf are unaffected by pullFactors.
     checkPullFactors(
         or(and(aRef, bRef),
@@ -778,19 +759,17 @@ public class RexProgramTest {
     final RexNode cRef = rexBuilder.makeFieldAccess(range, 2);
     final RexNode dRef = rexBuilder.makeFieldAccess(range, 3);
     final RexNode eRef = rexBuilder.makeFieldAccess(range, 4);
-    final RexLiteral true_ = rexBuilder.makeLiteral(true);
-    final RexLiteral false_ = rexBuilder.makeLiteral(false);
     final RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
 
     // and: remove duplicates
     checkSimplify(and(aRef, bRef, aRef), "AND(?0.a, ?0.b)");
 
     // and: remove true
-    checkSimplify(and(aRef, bRef, true_),
+    checkSimplify(and(aRef, bRef, trueLiteral),
         "AND(?0.a, ?0.b)");
 
     // and: false falsifies
-    checkSimplify(and(aRef, bRef, false_),
+    checkSimplify(and(aRef, bRef, falseLiteral),
         "false");
 
     // and: remove duplicate "not"s
@@ -798,7 +777,7 @@ public class RexProgramTest {
         "AND(?0.b, NOT(?0.a), NOT(?0.c))");
 
     // and: "not true" falsifies
-    checkSimplify(and(not(aRef), bRef, not(true_)),
+    checkSimplify(and(not(aRef), bRef, not(trueLiteral)),
         "false");
 
     // and: flatten and remove duplicates
@@ -817,32 +796,37 @@ public class RexProgramTest {
     checkSimplify(or(aRef, bRef, aRef), "OR(?0.a, ?0.b)");
 
     // or: remove false
-    checkSimplify(or(aRef, bRef, false_),
+    checkSimplify(or(aRef, bRef, falseLiteral),
         "OR(?0.a, ?0.b)");
 
     // or: true makes everything true
-    checkSimplify(or(aRef, bRef, true_), "true");
+    checkSimplify(or(aRef, bRef, trueLiteral), "true");
 
     // case: remove false branches
-    checkSimplify(case_(eq(bRef, cRef), dRef, false_, aRef, eRef),
+    checkSimplify(case_(eq(bRef, cRef), dRef, falseLiteral, aRef, eRef),
         "CASE(=(?0.b, ?0.c), ?0.d, ?0.e)");
 
     // case: true branches become the last branch
     checkSimplify(
-        case_(eq(bRef, cRef), dRef, true_, aRef, eq(cRef, dRef), eRef, cRef),
+        case_(eq(bRef, cRef), dRef, trueLiteral, aRef, eq(cRef, dRef), eRef, cRef),
         "CASE(=(?0.b, ?0.c), ?0.d, ?0.a)");
 
     // case: singleton
-    checkSimplify(case_(true_, aRef, eq(cRef, dRef), eRef, cRef), "?0.a");
+    checkSimplify(case_(trueLiteral, aRef, eq(cRef, dRef), eRef, cRef), "?0.a");
 
     // case: form an AND of branches that return true
     checkSimplify(
-        case_(aRef, true_, bRef, false_, cRef, false_, dRef, true_, false_),
+        case_(aRef, trueLiteral, bRef,
+            falseLiteral, cRef,
+            falseLiteral, dRef, trueLiteral,
+            falseLiteral),
         "OR(?0.a, AND(?0.d, NOT(?0.b), NOT(?0.c)))");
 
     checkSimplify(
-        case_(aRef, true_, bRef, false_, cRef, false_, dRef, true_, eRef,
-            false_, true_),
+        case_(aRef, trueLiteral, bRef,
+            falseLiteral, cRef,
+            falseLiteral, dRef, trueLiteral, eRef,
+            falseLiteral, trueLiteral),
         "OR(?0.a, AND(?0.d, NOT(?0.b), NOT(?0.c)), AND(NOT(?0.b), NOT(?0.c), NOT(?0.e)))");
 
     // is null, applied to not-null value
@@ -893,6 +877,78 @@ public class RexProgramTest {
     checkSimplifyFilter(and(lt(aRef, literal1), eq(aRef, literal1), ge(aRef, literal1)),
         "false");
   }
+
+  /** Unit test for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1289">[CALCITE-1289]
+   * RexUtil.simplifyCase() should account for nullability</a>. */
+  @Test public void testSimplifyCaseNotNullableBoolean() {
+    RexNode condition = eq(
+        rexBuilder.makeInputRef(
+            typeFactory.createTypeWithNullability(
+                typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
+            0),
+        rexBuilder.makeLiteral("S"));
+    RexCall caseNode = (RexCall) case_(condition, trueLiteral, falseLiteral);
+
+    RexCall result = (RexCall) RexUtil.simplify(rexBuilder, caseNode, false);
+    assertThat(result.getType().isNullable(), is(false));
+    assertThat(result.getType().getSqlTypeName(), is(SqlTypeName.BOOLEAN));
+    assertThat(result.getOperator(), is((SqlOperator) SqlStdOperatorTable.CASE));
+    assertThat(result.getOperands().size(), is((Object) 3));
+    assertThat(result.getOperands().get(0), is(condition));
+    assertThat(result.getOperands().get(1), is((RexNode) trueLiteral));
+    assertThat(result.getOperands().get(2), is((RexNode) falseLiteral));
+  }
+
+  @Test public void testSimplifyCaseNullableBoolean() {
+    RexNode condition = eq(
+        rexBuilder.makeInputRef(
+            typeFactory.createTypeWithNullability(
+                typeFactory.createSqlType(SqlTypeName.VARCHAR), false),
+            0),
+        rexBuilder.makeLiteral("S"));
+    RexCall caseNode = (RexCall) case_(condition, trueLiteral, falseLiteral);
+
+    RexCall result = (RexCall) RexUtil.simplify(rexBuilder, caseNode, false);
+    assertThat(result.getType().isNullable(), is(false));
+    assertThat(result.getType().getSqlTypeName(), is(SqlTypeName.BOOLEAN));
+    assertThat(result, is(condition));
+  }
+
+  @Test public void testSimplifyCaseNullableVarChar() {
+    RexNode condition = eq(
+        rexBuilder.makeInputRef(
+            typeFactory.createTypeWithNullability(
+                typeFactory.createSqlType(SqlTypeName.VARCHAR), false),
+            0),
+        rexBuilder.makeLiteral("S"));
+    RexLiteral aLiteral = rexBuilder.makeLiteral("A");
+    RexLiteral bLiteral = rexBuilder.makeLiteral("B");
+    RexCall caseNode = (RexCall) case_(condition, aLiteral, bLiteral);
+
+
+    RexCall result = (RexCall) RexUtil.simplify(rexBuilder, caseNode, false);
+    assertThat(result.getType().isNullable(), is(false));
+    assertThat(result.getType().getSqlTypeName(), is(SqlTypeName.CHAR));
+    assertThat(result, is(caseNode));
+  }
+
+  @Test public void testSimplifyAnd() {
+    RelDataType booleanNotNullableType =
+        typeFactory.createTypeWithNullability(
+            typeFactory.createSqlType(SqlTypeName.BOOLEAN), false);
+    RelDataType booleanNullableType =
+        typeFactory.createTypeWithNullability(
+            typeFactory.createSqlType(SqlTypeName.BOOLEAN), true);
+    RexNode andCondition =
+        and(rexBuilder.makeInputRef(booleanNotNullableType, 0),
+            rexBuilder.makeInputRef(booleanNullableType, 1),
+            rexBuilder.makeInputRef(booleanNotNullableType, 2));
+    RexNode result = RexUtil.simplify(rexBuilder, andCondition, false);
+    assertThat(result.getType().isNullable(), is(true));
+    assertThat(result.getType().getSqlTypeName(), is(SqlTypeName.BOOLEAN));
+  }
+
 }
 
 // End RexProgramTest.java