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/01/08 02:08:24 UTC

[1/6] calcite git commit: Fix variant of [CALCITE-923] that occurs in RelOptRulesTest.testPushFilterPastProject

Repository: calcite
Updated Branches:
  refs/heads/master 39f505fa9 -> 894dd9b52


Fix variant of [CALCITE-923] that occurs in RelOptRulesTest.testPushFilterPastProject


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

Branch: refs/heads/master
Commit: 808d540b3df47d98b25eeb83ee8f407909ba8c3a
Parents: 39f505f
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Dec 23 10:46:32 2015 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jan 5 15:16:34 2016 -0800

----------------------------------------------------------------------
 .../calcite/rel/rules/FilterJoinRule.java       |  2 +-
 .../org/apache/calcite/test/RelOptTestBase.java |  5 +++-
 .../apache/calcite/test/SqlToRelTestBase.java   | 24 ++++++++++----------
 3 files changed, 17 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/808d540b/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java
index 0a45a43..0bb3830 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java
@@ -268,7 +268,7 @@ public abstract class FilterJoinRule extends RelOptRule {
     // create a FilterRel on top of the join if needed
     relBuilder.filter(
         RexUtil.fixUp(rexBuilder, aboveFilters,
-            RelOptUtil.getFieldTypeList(newJoinRel.getRowType())));
+            RelOptUtil.getFieldTypeList(relBuilder.peek().getRowType())));
 
     call.transformTo(relBuilder.build());
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/808d540b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
index c6db96c..156a973 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
@@ -33,6 +33,7 @@ import com.google.common.collect.Lists;
 import java.util.List;
 
 import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
@@ -148,10 +149,11 @@ abstract class RelOptTestBase extends SqlToRelTestBase {
       relBefore = prePlanner.findBestExp();
     }
 
-    assertTrue(relBefore != null);
+    assertThat(relBefore, notNullValue());
 
     String planBefore = NL + RelOptUtil.toString(relBefore);
     diffRepos.assertEquals("planBefore", "${planBefore}", planBefore);
+    SqlToRelTestBase.assertValid(relBefore);
 
     planner.setRoot(relBefore);
     RelNode relAfter = planner.findBestExp();
@@ -166,6 +168,7 @@ abstract class RelOptTestBase extends SqlToRelTestBase {
             + "You must use unchanged=true or call checkPlanUnchanged");
       }
     }
+    SqlToRelTestBase.assertValid(relAfter);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/808d540b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
index eca9a55..e4d1394 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -114,6 +114,18 @@ public abstract class SqlToRelTestBase {
     return null;
   }
 
+  /**
+   * Checks that every node of a relational expression is valid.
+   *
+   * @param rel Relational expression
+   */
+  public static void assertValid(RelNode rel) {
+    SqlToRelConverterTest.RelValidityChecker checker =
+        new SqlToRelConverterTest.RelValidityChecker();
+    checker.go(rel);
+    assertEquals(0, checker.invalidCount);
+  }
+
   //~ Inner Interfaces -------------------------------------------------------
 
   /**
@@ -621,18 +633,6 @@ public abstract class SqlToRelTestBase {
       return new RelFieldTrimmer(getValidator(), relBuilder);
     }
 
-    /**
-     * Checks that every node of a relational expression is valid.
-     *
-     * @param rel Relational expression
-     */
-    protected void assertValid(RelNode rel) {
-      SqlToRelConverterTest.RelValidityChecker checker =
-          new SqlToRelConverterTest.RelValidityChecker();
-      checker.go(rel);
-      assertEquals(0, checker.invalidCount);
-    }
-
     public DiffRepository getDiffRepos() {
       return diffRepos;
     }


[5/6] calcite git commit: Fix RelToSqlConverterTest on Windows

Posted by jh...@apache.org.
Fix RelToSqlConverterTest on Windows


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

Branch: refs/heads/master
Commit: 894dd9b522228f43d26eba8fabcf244a33029d19
Parents: d4bbf58
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jan 5 15:51:45 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 6 02:37:31 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/894dd9b5/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
index 1d26eb8..8b19898 100644
--- a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
@@ -27,6 +27,7 @@ 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.util.Util;
 
 import org.junit.Test;
 
@@ -51,7 +52,7 @@ public class RelToSqlConverterTest {
       final RelToSqlConverter converter =
           new RelToSqlConverter(dialect);
       final SqlNode sqlNode = converter.visitChild(0, rel).asQuery();
-      assertThat(sqlNode.toSqlString(dialect).getSql(),
+      assertThat(Util.toLinux(sqlNode.toSqlString(dialect).getSql()),
           is(expectedQuery));
     } catch (Exception e) {
       assertTrue("Parsing failed throwing error: " + e.getMessage(), false);


[3/6] calcite git commit: [CALCITE-1042] Ensure that FILTER is BOOLEAN NOT NULL

Posted by jh...@apache.org.
[CALCITE-1042] Ensure that FILTER is BOOLEAN NOT NULL


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

Branch: refs/heads/master
Commit: efb668bb58cdc85e85facba3ec2365549778eee8
Parents: 904c73d
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jan 4 17:34:58 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 6 02:37:31 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/rel/core/Aggregate.java  | 13 ++++--
 .../apache/calcite/runtime/CalciteResource.java |  3 ++
 .../calcite/sql2rel/SqlToRelConverter.java      |  4 ++
 .../org/apache/calcite/tools/RelBuilder.java    |  8 ++++
 .../calcite/runtime/CalciteResource.properties  |  1 +
 .../org/apache/calcite/test/RelBuilderTest.java | 44 ++++++++++++++++++++
 .../java/org/apache/calcite/util/Smalls.java    | 23 ++++++++++
 core/src/test/resources/sql/agg.iq              |  2 +-
 .../linq4j/tree/DeterministicCodeOptimizer.java | 22 +++++-----
 9 files changed, 104 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java b/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
index 3a330d8..1a46ffd 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
@@ -140,12 +140,19 @@ public abstract class Aggregate extends SingleRel {
     assert groupSet.length() <= child.getRowType().getFieldCount();
     for (AggregateCall aggCall : aggCalls) {
       assert typeMatchesInferred(aggCall, true);
-      assert aggCall.filterArg < 0
-          || child.getRowType().getFieldList().get(aggCall.filterArg).getType()
-              .getSqlTypeName() == SqlTypeName.BOOLEAN;
+      Preconditions.checkArgument(aggCall.filterArg < 0
+          || isPredicate(child, aggCall.filterArg),
+          "filter must be BOOLEAN NOT NULL");
     }
   }
 
+  private boolean isPredicate(RelNode input, int index) {
+    final RelDataType type =
+        input.getRowType().getFieldList().get(index).getType();
+    return type.getSqlTypeName() == SqlTypeName.BOOLEAN
+        && !type.isNullable();
+  }
+
   /**
    * Creates an Aggregate by parsing serialized output.
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index 8c93081..ad3bc4f 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -607,6 +607,9 @@ public interface CalciteResource {
   @BaseMessage("Table ''{0}'' not found")
   ExInst<CalciteException> tableNotFound(String tableName);
 
+  @BaseMessage("FILTER expression must be of type BOOLEAN")
+  ExInst<CalciteException> filterMustBeBoolean();
+
   @BaseMessage("Cannot stream results of a query with no streaming inputs: ''{0}''. At least one input should be convertible to a stream")
   ExInst<SqlValidatorException> cannotStreamResultsForNonStreamingInputs(String inputs);
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/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 2e87b53..3b402af 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -4506,6 +4506,10 @@ public class SqlToRelConverter {
         if (filter != null) {
           RexNode convertedExpr = bb.convertExpression(filter);
           assert convertedExpr != null;
+          if (convertedExpr.getType().isNullable()) {
+            convertedExpr =
+                rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, convertedExpr);
+          }
           filterArg = lookupOrCreateGroupExpr(convertedExpr);
         }
       } finally {

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
index 2cd786c..252e703 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -600,6 +600,14 @@ public class RelBuilder {
   /** Creates a call to an aggregate function. */
   public AggCall aggregateCall(SqlAggFunction aggFunction, boolean distinct,
       RexNode filter, String alias, Iterable<? extends RexNode> operands) {
+    if (filter != null) {
+      if (filter.getType().getSqlTypeName() != SqlTypeName.BOOLEAN) {
+        throw Static.RESOURCE.filterMustBeBoolean().ex();
+      }
+      if (filter.getType().isNullable()) {
+        filter = call(SqlStdOperatorTable.IS_TRUE, filter);
+      }
+    }
     return new AggCallImpl(aggFunction, distinct, filter, alias,
         ImmutableList.copyOf(operands));
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index 85dcc1d..d050f85 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -198,5 +198,6 @@ ModifiableViewMustBeBasedOnSingleTable=Modifiable view must be based on a single
 MoreThanOneMappedColumn=View is not modifiable. More than one expression maps to column ''{0}'' of base table ''{1}''
 NoValueSuppliedForViewColumn=View is not modifiable. No value is supplied for NOT NULL column ''{0}'' of base table ''{1}''
 TableNotFound=Table ''{0}'' not found
+FilterMustBeBoolean=FILTER expression must be of type BOOLEAN
 CannotStreamResultsForNonStreamingInputs=Cannot stream results of a query with no streaming inputs: ''{0}''. At least one input should be convertible to a stream
 # End CalciteResource.properties

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
index 82757ab..7df8dcf 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -30,6 +30,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParser;
@@ -426,6 +427,49 @@ public class RelBuilderTest {
     assertThat(str(root), is(expected));
   }
 
+  @Test public void testAggregateFilterFails() {
+    // Equivalent SQL:
+    //   SELECT deptno, SUM(sal) FILTER (WHERE comm) AS c
+    //   FROM emp
+    //   GROUP BY deptno
+    try {
+      final RelBuilder builder = RelBuilder.create(config().build());
+      RelNode root =
+          builder.scan("EMP")
+              .aggregate(
+                  builder.groupKey(builder.field("DEPTNO")),
+                  builder.aggregateCall(SqlStdOperatorTable.SUM, false,
+                      builder.field("COMM"), "C", builder.field("SAL")))
+              .build();
+      fail("expected error, got " + root);
+    } catch (CalciteException e) {
+      assertThat(e.getMessage(),
+          is("FILTER expression must be of type BOOLEAN"));
+    }
+  }
+
+  @Test public void testAggregateFilterNullable() {
+    // Equivalent SQL:
+    //   SELECT deptno, SUM(sal) FILTER (WHERE comm < 100) AS c
+    //   FROM emp
+    //   GROUP BY deptno
+    final RelBuilder builder = RelBuilder.create(config().build());
+    RelNode root =
+        builder.scan("EMP")
+            .aggregate(
+                builder.groupKey(builder.field("DEPTNO")),
+                builder.aggregateCall(SqlStdOperatorTable.SUM, false,
+                    builder.call(SqlStdOperatorTable.LESS_THAN,
+                        builder.field("COMM"), builder.literal(100)), "C",
+                    builder.field("SAL")))
+            .build();
+    final String expected = ""
+        + "LogicalAggregate(group=[{7}], C=[SUM($5) FILTER $8])\n"
+        + "  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], $f8=[IS TRUE(<($6, 100))])\n"
+        + "    LogicalTableScan(table=[[scott, EMP]])\n";
+    assertThat(str(root), is(expected));
+  }
+
   @Test public void testAggregateGroupingKeyOutOfRangeFails() {
     final RelBuilder builder = RelBuilder.create(config().build());
     try {

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/core/src/test/java/org/apache/calcite/util/Smalls.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/Smalls.java b/core/src/test/java/org/apache/calcite/util/Smalls.java
index fe04094..62cc064 100644
--- a/core/src/test/java/org/apache/calcite/util/Smalls.java
+++ b/core/src/test/java/org/apache/calcite/util/Smalls.java
@@ -24,6 +24,7 @@ import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.linq4j.Linq4j;
 import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.function.Deterministic;
 import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.linq4j.function.Function2;
 import org.apache.calcite.linq4j.function.Parameter;
@@ -369,6 +370,15 @@ public class Smalls {
   }
 
   /** See {@link CountArgs0Function}. */
+  public abstract static class CountArgs1NullableFunction {
+    private CountArgs1NullableFunction() {}
+
+    public static int eval(Short x) {
+      return -1;
+    }
+  }
+
+  /** See {@link CountArgs0Function}. */
   public abstract static class CountArgs2Function {
     private CountArgs2Function() {}
 
@@ -404,12 +414,25 @@ public class Smalls {
   }
 
   /** UDF class that provides user-defined functions for each data type. */
+  @Deterministic
   public static class AllTypesFunction {
     private AllTypesFunction() {}
 
     public static long dateFun(java.sql.Date x) { return x == null ? -1L : x.getTime(); }
     public static long timestampFun(java.sql.Timestamp x) { return x == null ? -1L : x.getTime(); }
     public static long timeFun(java.sql.Time x) { return x == null ? -1L : x.getTime(); }
+
+    public static java.sql.Date toDateFun(int x) { return new java.sql.Date(x); }
+
+    public static java.sql.Date toDateFun(Long x) {
+      return x == null ? null : new java.sql.Date(x);
+    }
+    public static java.sql.Timestamp toTimestampFun(Long x) {
+      return x == null ? null : new java.sql.Timestamp(x);
+    }
+    public static java.sql.Time toTimeFun(Long x) {
+      return x == null ? null : new java.sql.Time(x);
+    }
   }
 
   /** Example of a user-defined aggregate function (UDAF). */

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/core/src/test/resources/sql/agg.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/agg.iq b/core/src/test/resources/sql/agg.iq
index 720b3df..b76cc90 100644
--- a/core/src/test/resources/sql/agg.iq
+++ b/core/src/test/resources/sql/agg.iq
@@ -842,7 +842,7 @@ group by deptno;
 
 !ok
 EnumerableAggregate(group=[{0}], CF=[COUNT() FILTER $1], C=[COUNT()])
-  EnumerableCalc(expr#0..1=[{inputs}], expr#2=['CLERK'], expr#3=[=($t0, $t2)], DEPTNO=[$t1], $f1=[$t3])
+  EnumerableCalc(expr#0..1=[{inputs}], expr#2=['CLERK'], expr#3=[=($t0, $t2)], expr#4=[IS TRUE($t3)], DEPTNO=[$t1], $f1=[$t4])
     EnumerableUnion(all=[true])
       EnumerableCalc(expr#0..7=[{inputs}], expr#8=[20], expr#9=[<($t7, $t8)], JOB=[$t2], DEPTNO=[$t7], $condition=[$t9])
         EnumerableTableScan(table=[[scott, EMP]])

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java
index f161c43..e117d28 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java
@@ -19,13 +19,13 @@ package org.apache.calcite.linq4j.tree;
 import org.apache.calcite.linq4j.function.Deterministic;
 import org.apache.calcite.linq4j.function.NonDeterministic;
 
+import com.google.common.collect.ImmutableSet;
+
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.math.BigDecimal;
 import java.math.BigInteger;
-import java.util.Arrays;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
@@ -46,22 +46,20 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
    * For instance, cast expression will not be factored to a field,
    * but we still need to track its constant status.
    */
-  protected final Map<Expression, Boolean> constants =
-      new IdentityHashMap<Expression, Boolean>();
+  protected final Map<Expression, Boolean> constants = new IdentityHashMap<>();
 
   /**
    * The map that de-duplicates expressions, so the same expressions may reuse
    * the same final static fields.
    */
-  protected final Map<Expression, ParameterExpression> dedup =
-      new HashMap<Expression, ParameterExpression>();
+  protected final Map<Expression, ParameterExpression> dedup = new HashMap<>();
 
   /**
    * The map of all the added final static fields. Allows to identify if the
    * name is occupied or not.
    */
   protected final Map<String, ParameterExpression> fieldsByName =
-      new HashMap<String, ParameterExpression>();
+      new HashMap<>();
 
   // Pre-compiled patterns for generation names for the final static fields
   private static final Pattern NON_ASCII = Pattern.compile("[^0-9a-zA-Z$]+");
@@ -70,10 +68,9 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
       Pattern.compile(Pattern.quote(FIELD_PREFIX));
 
   private static final Set<Class> DETERMINISTIC_CLASSES =
-      new HashSet<Class>(
-          Arrays.<Class>asList(Byte.class, Boolean.class, Short.class,
-              Integer.class, Long.class, BigInteger.class, BigDecimal.class,
-              String.class, Math.class));
+      ImmutableSet.<Class>of(Byte.class, Boolean.class, Short.class,
+          Integer.class, Long.class, BigInteger.class, BigDecimal.class,
+          String.class, Math.class);
 
   /**
    * Creates a child optimizer.
@@ -337,7 +334,8 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
    */
   protected boolean allMethodsDeterministic(Class klass) {
     return DETERMINISTIC_CLASSES.contains(klass)
-           || klass.isAnnotationPresent(Deterministic.class);
+        || klass.getCanonicalName().equals("org.apache.calcite.avatica.util.DateTimeUtils")
+        || klass.isAnnotationPresent(Deterministic.class);
   }
 
   /**


[2/6] calcite git commit: [CALCITE-1039] Assign a SqlKind value for each built-in aggregate function

Posted by jh...@apache.org.
[CALCITE-1039] Assign a SqlKind value for each built-in aggregate function

Also, deprecate getParameterTypes and getReturnType methods in several aggregate functions.


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

Branch: refs/heads/master
Commit: 8531ab11715109716215c0006d28014754e566f9
Parents: 808d540
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jan 4 02:47:17 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jan 5 15:16:35 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/plan/RelOptUtil.java     | 22 +----
 .../org/apache/calcite/sql/SqlAggFunction.java  | 14 +++
 .../java/org/apache/calcite/sql/SqlKind.java    | 97 ++++++++++++++++++--
 .../org/apache/calcite/sql/SqlRankFunction.java | 17 ----
 .../calcite/sql/fun/SqlAvgAggFunction.java      | 48 ++++------
 .../calcite/sql/fun/SqlCountAggFunction.java    |  2 +-
 .../calcite/sql/fun/SqlCovarAggFunction.java    | 38 +++-----
 .../sql/fun/SqlFirstLastValueAggFunction.java   | 14 ++-
 .../sql/fun/SqlHistogramAggFunction.java        | 10 +-
 .../calcite/sql/fun/SqlLeadLagAggFunction.java  | 23 +++--
 .../calcite/sql/fun/SqlMinMaxAggFunction.java   | 49 ++++++----
 .../calcite/sql/fun/SqlNtileAggFunction.java    | 13 +--
 .../sql/fun/SqlSingleValueAggFunction.java      |  4 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    | 52 +++++------
 .../calcite/sql/fun/SqlSumAggFunction.java      |  4 +-
 .../sql/fun/SqlSumEmptyIsZeroAggFunction.java   |  2 +-
 16 files changed, 229 insertions(+), 180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/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 a7b3e08..e658749 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -64,11 +64,9 @@ import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
-import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.fun.SqlMinMaxAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.MultisetSqlType;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -376,18 +374,9 @@ public abstract class RelOptUtil {
       // doing a min(TRUE)
 
       ret = createProject(ret, ImmutableList.of(extraExpr), null);
-      final List<RelDataType> argTypes =
-          ImmutableList.of(
-              typeFactory.createSqlType(SqlTypeName.BOOLEAN));
-
-      SqlAggFunction minFunction =
-          new SqlMinMaxAggFunction(
-              argTypes,
-              true,
-              SqlMinMaxAggFunction.MINMAX_COMPARABLE);
 
       final AggregateCall aggCall =
-          AggregateCall.create(minFunction,
+          AggregateCall.create(SqlStdOperatorTable.MIN,
               false,
               ImmutableList.of(0),
               -1,
@@ -457,15 +446,8 @@ public abstract class RelOptUtil {
 
       ret = createProject(ret, exprs, null);
 
-      final List<RelDataType> argTypes =
-          ImmutableList.of(typeFactory.createSqlType(SqlTypeName.BOOLEAN));
-
-      final SqlAggFunction minFunction =
-          new SqlMinMaxAggFunction(argTypes, true,
-              SqlMinMaxAggFunction.MINMAX_COMPARABLE);
-
       final AggregateCall aggCall =
-          AggregateCall.create(minFunction,
+          AggregateCall.create(SqlStdOperatorTable.MIN,
               false,
               ImmutableList.of(projectedKeyCount),
               -1,

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
index 83fe248..d58987d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
@@ -17,12 +17,16 @@
 package org.apache.calcite.sql;
 
 import org.apache.calcite.plan.Context;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 
+import java.util.List;
+
 /**
  * Abstract base class for the definition of an aggregate function: an operator
  * which aggregates sets of values into a result.
@@ -119,6 +123,16 @@ public abstract class SqlAggFunction extends SqlFunction implements Context {
   public final boolean requiresOver() {
     return requiresOver;
   }
+
+  @Deprecated // to be removed before 2.0
+  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
+    throw new UnsupportedOperationException("remove before calcite-2.0");
+  }
+
+  @Deprecated // to be removed before 2.0
+  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
+    throw new UnsupportedOperationException("remove before calcite-2.0");
+  }
 }
 
 // End SqlAggFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/SqlKind.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index 5a8a2fa..78a9d7d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -625,6 +625,74 @@ public enum SqlKind {
   /** The internal {@code GROUP_ID()} function. */
   GROUP_ID,
 
+  // Aggregate functions
+
+  /** The {@code COUNT} aggregate function. */
+  COUNT,
+
+  /** The {@code SUM} aggregate function. */
+  SUM,
+
+  /** The {@code SUM0} aggregate function. */
+  SUM0,
+
+  /** The {@code MIN} aggregate function. */
+  MIN,
+
+  /** The {@code MAX} aggregate function. */
+  MAX,
+
+  /** The {@code LEAD} aggregate function. */
+  LEAD,
+
+  /** The {@code LAG} aggregate function. */
+  LAG,
+
+  /** The {@code FIRST_VALUE} aggregate function. */
+  FIRST_VALUE,
+
+  /** The {@code LAST_VALUE} aggregate function. */
+  LAST_VALUE,
+
+  /** The {@code COVAR_POP} aggregate function. */
+  COVAR_POP,
+
+  /** The {@code COVAR_SAMP} aggregate function. */
+  COVAR_SAMP,
+
+  /** The {@code REGR_SXX} aggregate function. */
+  REGR_SXX,
+
+  /** The {@code REGR_SYY} aggregate function. */
+  REGR_SYY,
+
+  /** The {@code AVG} aggregate function. */
+  AVG,
+
+  /** The {@code STDDEV_POP} aggregate function. */
+  STDDEV_POP,
+
+  /** The {@code STDDEV_SAMP} aggregate function. */
+  STDDEV_SAMP,
+
+  /** The {@code VAR_POP} aggregate function. */
+  VAR_POP,
+
+  /** The {@code VAR_SAMP} aggregate function. */
+  VAR_SAMP,
+
+  /** The {@code NTILE} aggregate function. */
+  NTILE,
+
+  /** The {@code COLLECT} aggregate function. */
+  COLLECT,
+
+  /** The {@code FUSION} aggregate function. */
+  FUSION,
+
+  /** The {@code SINGLE_VALUE} aggregate function. */
+  SINGLE_VALUE,
+
   // DDL and session control statements follow. The list is not exhaustive: feel
   // free to add more.
 
@@ -699,10 +767,21 @@ public enum SqlKind {
       EnumSet.of(UNION, INTERSECT, EXCEPT);
 
   /**
+   * Category consisting of all built-in aggregate functions.
+   */
+  public static final EnumSet<SqlKind> AGGREGATE =
+      EnumSet.of(COUNT, SUM, SUM0, MIN, MAX, LEAD, LAG, FIRST_VALUE,
+          LAST_VALUE, COVAR_POP, COVAR_SAMP, REGR_SXX, REGR_SYY,
+          AVG, STDDEV_POP, STDDEV_SAMP, VAR_POP, VAR_SAMP, NTILE, COLLECT,
+          FUSION, SINGLE_VALUE);
+
+  /**
    * Category consisting of all expression operators.
    *
    * <p>A node is an expression if it is NOT one of the following:
    * {@link #AS},
+   * {@link #ARGUMENT_ASSIGNMENT},
+   * {@link #DEFAULT},
    * {@link #DESCENDING},
    * {@link #SELECT},
    * {@link #JOIN},
@@ -715,17 +794,19 @@ public enum SqlKind {
    * {@link #FOLLOWING},
    * {@link #ORDER_BY},
    * {@link #COLLECTION_TABLE},
-   * {@link #TABLESAMPLE}.
+   * {@link #TABLESAMPLE},
+   * or an aggregate function.
    */
   public static final Set<SqlKind> EXPRESSION =
       EnumSet.complementOf(
-          EnumSet.of(
-              AS, ARGUMENT_ASSIGNMENT, DEFAULT,
-              DESCENDING, CUBE, ROLLUP, GROUPING_SETS, EXTEND,
-              SELECT, JOIN, OTHER_FUNCTION, CAST, TRIM, FLOOR, CEIL,
-              LITERAL_CHAIN, JDBC_FN, PRECEDING, FOLLOWING, ORDER_BY,
-              NULLS_FIRST, NULLS_LAST, COLLECTION_TABLE, TABLESAMPLE,
-              VALUES, WITH, WITH_ITEM));
+          concat(
+              EnumSet.of(AS, ARGUMENT_ASSIGNMENT, DEFAULT,
+                  DESCENDING, CUBE, ROLLUP, GROUPING_SETS, EXTEND,
+                  SELECT, JOIN, OTHER_FUNCTION, CAST, TRIM, FLOOR, CEIL,
+                  LITERAL_CHAIN, JDBC_FN, PRECEDING, FOLLOWING, ORDER_BY,
+                  NULLS_FIRST, NULLS_LAST, COLLECTION_TABLE, TABLESAMPLE,
+                  VALUES, WITH, WITH_ITEM),
+              AGGREGATE));
 
   /**
    * Category consisting of all DML operators.

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
index e2a2490..738add1 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
@@ -16,23 +16,13 @@
  */
 package org.apache.calcite.sql;
 
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
-
 /**
  * Operator which aggregates sets of values into a result.
  */
 public class SqlRankFunction extends SqlAggFunction {
-  //~ Instance fields --------------------------------------------------------
-
-  private final RelDataType type = null;
-
   //~ Constructors -----------------------------------------------------------
 
   public SqlRankFunction(String name, boolean requiresOrder) {
@@ -54,13 +44,6 @@ public class SqlRankFunction extends SqlAggFunction {
     return false;
   }
 
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    return type;
-  }
-
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    return ImmutableList.of(type);
-  }
 }
 
 // End SqlRankFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
index b86d49c..95f8049 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
@@ -17,16 +17,13 @@
 package org.apache.calcite.sql.fun;
 
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
+import com.google.common.base.Preconditions;
 
 /**
  * <code>Avg</code> is an aggregator which returns the average of the values
@@ -35,56 +32,49 @@ import java.util.List;
  * double</code>), and the result is the same type.
  */
 public class SqlAvgAggFunction extends SqlAggFunction {
-  //~ Instance fields --------------------------------------------------------
-
-  private final RelDataType type;
-  private final Subtype subtype;
-
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a SqlAvgAggFunction
-   *
-   * @param type    Data type
-   * @param subtype Specific function, e.g. AVG or STDDEV_POP
+   * Creates a SqlAvgAggFunction.
    */
-  public SqlAvgAggFunction(
-      RelDataType type,
-      Subtype subtype) {
-    super(
-        subtype.name(),
+  public SqlAvgAggFunction(SqlKind kind) {
+    super(kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.NUMERIC,
         SqlFunctionCategory.NUMERIC,
         false,
         false);
-    this.type = type;
-    this.subtype = subtype;
+    Preconditions.checkArgument(kind == SqlKind.AVG
+        || kind == SqlKind.STDDEV_POP
+        || kind == SqlKind.STDDEV_SAMP
+        || kind == SqlKind.VAR_POP
+        || kind == SqlKind.VAR_SAMP);
   }
 
-  //~ Methods ----------------------------------------------------------------
-
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    return ImmutableList.of(type);
+  @Deprecated // to be removed before 2.0
+  public SqlAvgAggFunction(
+      RelDataType type,
+      Subtype subtype) {
+    this(SqlKind.valueOf(subtype.name()));
   }
 
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    return type;
-  }
+    //~ Methods ----------------------------------------------------------------
 
   /**
    * Returns the specific function, e.g. AVG or STDDEV_POP.
    *
    * @return Subtype
    */
+  @Deprecated // to be removed before 2.0
   public Subtype getSubtype() {
-    return subtype;
+    return Subtype.valueOf(kind.name());
   }
 
   /** Sub-type of aggregate function. */
+  @Deprecated // to be removed before 2.0
   public enum Subtype {
     AVG,
     STDDEV_POP,

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
index e91f30b..dac4b21 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
@@ -48,7 +48,7 @@ public class SqlCountAggFunction extends SqlAggFunction {
     super(
         "COUNT",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.COUNT,
         ReturnTypes.BIGINT,
         null,
         SqlValidator.STRICT

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
index df36095..ea23300 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
@@ -17,16 +17,13 @@
 package org.apache.calcite.sql.fun;
 
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
+import com.google.common.base.Preconditions;
 
 /**
  * <code>Covar</code> is an aggregator which returns the Covariance of the
@@ -37,53 +34,48 @@ import java.util.List;
 public class SqlCovarAggFunction extends SqlAggFunction {
   //~ Instance fields --------------------------------------------------------
 
-  private final RelDataType type;
-  private final Subtype subtype;
-
   //~ Constructors -----------------------------------------------------------
 
   /**
    * Creates a SqlCovarAggFunction.
-   *
-   * @param type    Data type
-   * @param subtype Specific function, e.g. COVAR_POP
    */
-  public SqlCovarAggFunction(RelDataType type, Subtype subtype) {
-    super(subtype.name(),
+  public SqlCovarAggFunction(SqlKind kind) {
+    super(kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.NUMERIC_NUMERIC,
         SqlFunctionCategory.NUMERIC,
         false,
         false);
-    this.type = type;
-    this.subtype = subtype;
+    Preconditions.checkArgument(kind == SqlKind.COVAR_POP
+        || kind == SqlKind.COVAR_SAMP
+        || kind == SqlKind.REGR_SXX
+        || kind == SqlKind.REGR_SYY);
   }
 
-  //~ Methods ----------------------------------------------------------------
-
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    return ImmutableList.of(type);
+  @Deprecated // to be removed before 2.0
+  public SqlCovarAggFunction(RelDataType type, Subtype subtype) {
+    this(SqlKind.valueOf(subtype.name()));
   }
 
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    return type;
-  }
+  //~ Methods ----------------------------------------------------------------
 
   /**
    * Returns the specific function, e.g. COVAR_POP or COVAR_SAMP.
    *
    * @return Subtype
    */
+  @Deprecated // to be removed before 2.0
   public Subtype getSubtype() {
-    return subtype;
+    return Subtype.valueOf(kind.name());
   }
 
   /**
    * Enum for defining specific types.
    */
+  @Deprecated // to be removed before 2.0
   public enum Subtype {
     COVAR_POP,
     COVAR_SAMP,

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
index bc75424..edc6499 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
@@ -25,6 +25,7 @@ import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeName;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
@@ -37,17 +38,24 @@ import java.util.List;
 public class SqlFirstLastValueAggFunction extends SqlAggFunction {
   //~ Constructors -----------------------------------------------------------
 
-  public SqlFirstLastValueAggFunction(boolean firstFlag) {
+  public SqlFirstLastValueAggFunction(SqlKind kind) {
     super(
-        firstFlag ? "FIRST_VALUE" : "LAST_VALUE",
+        kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.ANY,
         SqlFunctionCategory.NUMERIC,
         false,
         true);
+    Preconditions.checkArgument(kind == SqlKind.FIRST_VALUE
+        || kind == SqlKind.LAST_VALUE);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public SqlFirstLastValueAggFunction(boolean firstFlag) {
+    this(firstFlag ? SqlKind.FIRST_VALUE : SqlKind.LAST_VALUE);
   }
 
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
index 8572338..1ba3821 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
@@ -29,15 +29,16 @@ import com.google.common.collect.ImmutableList;
 import java.util.List;
 
 /**
- * <code>HistogramAgg</code> is base operator which supports the Histogram
- * MIN/MAX aggregatoin functions. which returns the sum of the values which go
+ * <code>HISTOGRAM</code> is the base operator that supports the Histogram
+ * MIN/MAX aggregate functions. It returns the sum of the values which go
  * into it. It has precisely one argument of numeric type (<code>int</code>,
- * <code>long</code>, <code>float</code>, <code>double</code>) results are
- * retrived with (<code>HistogramMin</code>) and (<code>HistogramMax</code>)
+ * <code>long</code>, <code>float</code>, <code>double</code>); results are
+ * retrieved using (<code>HistogramMin</code>) and (<code>HistogramMax</code>).
  */
 public class SqlHistogramAggFunction extends SqlAggFunction {
   //~ Instance fields --------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   private final RelDataType type;
 
   //~ Constructors -----------------------------------------------------------
@@ -62,6 +63,7 @@ public class SqlHistogramAggFunction extends SqlAggFunction {
     return ImmutableList.of(type);
   }
 
+  @Deprecated // to be removed before 2.0
   public RelDataType getType() {
     return type;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
index a2b0c45..5e5498b 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
@@ -17,7 +17,6 @@
 package org.apache.calcite.sql.fun;
 
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
@@ -31,6 +30,7 @@ import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeTransform;
 import org.apache.calcite.sql.type.SqlTypeTransforms;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
@@ -73,30 +73,29 @@ public class SqlLeadLagAggFunction extends SqlAggFunction {
         }
       });
 
-  public SqlLeadLagAggFunction(boolean isLead) {
-    super(
-        isLead ? "LEAD" : "LAG",
+  public SqlLeadLagAggFunction(SqlKind kind) {
+    super(kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         RETURN_TYPE,
         null,
         OPERAND_TYPES,
         SqlFunctionCategory.NUMERIC,
         false,
         true);
+    Preconditions.checkArgument(kind == SqlKind.LEAD
+        || kind == SqlKind.LAG);
   }
 
-  @Override public boolean allowsFraming() {
-    return false;
+  @Deprecated // to be removed before 2.0
+  public SqlLeadLagAggFunction(boolean isLead) {
+    this(isLead ? SqlKind.LEAD : SqlKind.LAG);
   }
 
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    throw new UnsupportedOperationException("remove before calcite-0.9");
+  @Override public boolean allowsFraming() {
+    return false;
   }
 
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    throw new UnsupportedOperationException("remove before calcite-0.9");
-  }
 }
 
 // End SqlLeadLagAggFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
index 27a1a8b..391c09a 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
@@ -26,6 +26,9 @@ import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
 import java.util.List;
 
 /**
@@ -57,62 +60,72 @@ public class SqlMinMaxAggFunction extends SqlAggFunction {
 
   //~ Instance fields --------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   public final List<RelDataType> argTypes;
-  private final boolean isMin;
-  private final int kind;
+  private final int minMaxKind;
 
   //~ Constructors -----------------------------------------------------------
 
-  public SqlMinMaxAggFunction(
-      List<RelDataType> argTypes,
-      boolean isMin,
-      int kind) {
-    super(
-        isMin ? "MIN" : "MAX",
+  /** Creates a SqlMinMaxAggFunction. */
+  public SqlMinMaxAggFunction(SqlKind kind) {
+    super(kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.COMPARABLE_ORDERED,
         SqlFunctionCategory.SYSTEM,
         false,
         false);
-    this.argTypes = argTypes;
-    this.isMin = isMin;
-    this.kind = kind;
+    this.argTypes = ImmutableList.of();
+    this.minMaxKind = MINMAX_COMPARABLE;
+    Preconditions.checkArgument(kind == SqlKind.MIN
+        || kind == SqlKind.MAX);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public SqlMinMaxAggFunction(
+      List<RelDataType> argTypes,
+      boolean isMin,
+      int minMaxKind) {
+    this(isMin ? SqlKind.MIN : SqlKind.MAX);
+    assert argTypes.isEmpty();
+    assert minMaxKind == MINMAX_COMPARABLE;
   }
 
   //~ Methods ----------------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   public boolean isMin() {
-    return isMin;
+    return kind == SqlKind.MIN;
   }
 
+  @Deprecated // to be removed before 2.0
   public int getMinMaxKind() {
-    return kind;
+    return minMaxKind;
   }
 
   public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    switch (kind) {
+    switch (minMaxKind) {
     case MINMAX_PRIMITIVE:
     case MINMAX_COMPARABLE:
       return argTypes;
     case MINMAX_COMPARATOR:
       return argTypes.subList(1, 2);
     default:
-      throw Util.newInternal("bad kind: " + kind);
+      throw Util.newInternal("bad kind: " + minMaxKind);
     }
   }
 
   public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    switch (kind) {
+    switch (minMaxKind) {
     case MINMAX_PRIMITIVE:
     case MINMAX_COMPARABLE:
       return argTypes.get(0);
     case MINMAX_COMPARATOR:
       return argTypes.get(1);
     default:
-      throw Util.newInternal("bad kind: " + kind);
+      throw Util.newInternal("bad kind: " + minMaxKind);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
index 524e7d6..55eb8cb 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
@@ -16,16 +16,12 @@
  */
 package org.apache.calcite.sql.fun;
 
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
-import java.util.List;
-
 /**
  * <code>NTILE</code> aggregate function
  * return the value of given expression evaluated at given offset.
@@ -35,7 +31,7 @@ public class SqlNtileAggFunction extends SqlAggFunction {
     super(
         "NTILE",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.NTILE,
         ReturnTypes.INTEGER,
         null,
         OperandTypes.POSITIVE_INTEGER_LITERAL,
@@ -44,13 +40,6 @@ public class SqlNtileAggFunction extends SqlAggFunction {
         true);
   }
 
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    throw new UnsupportedOperationException("remove before calcite-0.9");
-  }
-
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    throw new UnsupportedOperationException("remove before calcite-0.9");
-  }
 }
 
 // End SqlNtileAggFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
index a27e180..c0c977c 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
@@ -35,6 +35,7 @@ import java.util.List;
 public class SqlSingleValueAggFunction extends SqlAggFunction {
   //~ Instance fields --------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   private final RelDataType type;
 
   //~ Constructors -----------------------------------------------------------
@@ -44,7 +45,7 @@ public class SqlSingleValueAggFunction extends SqlAggFunction {
     super(
         "SINGLE_VALUE",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.SINGLE_VALUE,
         ReturnTypes.ARG0,
         null,
         OperandTypes.ANY,
@@ -64,6 +65,7 @@ public class SqlSingleValueAggFunction extends SqlAggFunction {
     return type;
   }
 
+  @Deprecated // to be removed before 2.0
   public RelDataType getType() {
     return type;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index fc524c6..3311f50 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -16,7 +16,6 @@
  */
 package org.apache.calcite.sql.fun;
 
-import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlAsOperator;
 import org.apache.calcite.sql.SqlBinaryOperator;
@@ -51,8 +50,6 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable;
 import org.apache.calcite.sql.validate.SqlModality;
 
-import com.google.common.collect.ImmutableList;
-
 /**
  * Implementation of {@link org.apache.calcite.sql.SqlOperatorTable} containing
  * the standard operators and functions.
@@ -688,43 +685,37 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    * <code>MIN</code> aggregate function.
    */
   public static final SqlAggFunction MIN =
-      new SqlMinMaxAggFunction(
-          ImmutableList.<RelDataType>of(),
-          true,
-          SqlMinMaxAggFunction.MINMAX_COMPARABLE);
+      new SqlMinMaxAggFunction(SqlKind.MIN);
 
   /**
    * <code>MAX</code> aggregate function.
    */
   public static final SqlAggFunction MAX =
-      new SqlMinMaxAggFunction(
-          ImmutableList.<RelDataType>of(),
-          false,
-          SqlMinMaxAggFunction.MINMAX_COMPARABLE);
+      new SqlMinMaxAggFunction(SqlKind.MAX);
 
   /**
    * <code>LAST_VALUE</code> aggregate function.
    */
   public static final SqlAggFunction LAST_VALUE =
-      new SqlFirstLastValueAggFunction(false);
+      new SqlFirstLastValueAggFunction(SqlKind.LAST_VALUE);
 
   /**
    * <code>FIRST_VALUE</code> aggregate function.
    */
   public static final SqlAggFunction FIRST_VALUE =
-      new SqlFirstLastValueAggFunction(true);
+      new SqlFirstLastValueAggFunction(SqlKind.FIRST_VALUE);
 
   /**
    * <code>LEAD</code> aggregate function.
    */
   public static final SqlAggFunction LEAD =
-      new SqlLeadLagAggFunction(true);
+      new SqlLeadLagAggFunction(SqlKind.LEAD);
 
   /**
    * <code>LAG</code> aggregate function.
    */
   public static final SqlAggFunction LAG =
-      new SqlLeadLagAggFunction(false);
+      new SqlLeadLagAggFunction(SqlKind.LAG);
 
   /**
    * <code>NTILE</code> aggregate function.
@@ -742,55 +733,55 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    * <code>AVG</code> aggregate function.
    */
   public static final SqlAggFunction AVG =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.AVG);
+      new SqlAvgAggFunction(SqlKind.AVG);
 
   /**
    * <code>STDDEV_POP</code> aggregate function.
    */
   public static final SqlAggFunction STDDEV_POP =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.STDDEV_POP);
+      new SqlAvgAggFunction(SqlKind.STDDEV_POP);
 
   /**
    * <code>REGR_SXX</code> aggregate function.
    */
   public static final SqlAggFunction REGR_SXX =
-      new SqlCovarAggFunction(null, SqlCovarAggFunction.Subtype.REGR_SXX);
+      new SqlCovarAggFunction(SqlKind.REGR_SXX);
 
   /**
    * <code>REGR_SYY</code> aggregate function.
    */
   public static final SqlAggFunction REGR_SYY =
-      new SqlCovarAggFunction(null, SqlCovarAggFunction.Subtype.REGR_SYY);
+      new SqlCovarAggFunction(SqlKind.REGR_SYY);
 
   /**
    * <code>COVAR_POP</code> aggregate function.
    */
   public static final SqlAggFunction COVAR_POP =
-      new SqlCovarAggFunction(null, SqlCovarAggFunction.Subtype.COVAR_POP);
+      new SqlCovarAggFunction(SqlKind.COVAR_POP);
 
   /**
    * <code>COVAR_SAMP</code> aggregate function.
    */
   public static final SqlAggFunction COVAR_SAMP =
-      new SqlCovarAggFunction(null, SqlCovarAggFunction.Subtype.COVAR_SAMP);
+      new SqlCovarAggFunction(SqlKind.COVAR_SAMP);
 
   /**
    * <code>STDDEV_SAMP</code> aggregate function.
    */
   public static final SqlAggFunction STDDEV_SAMP =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.STDDEV_SAMP);
+      new SqlAvgAggFunction(SqlKind.STDDEV_SAMP);
 
   /**
    * <code>VAR_POP</code> aggregate function.
    */
   public static final SqlAggFunction VAR_POP =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.VAR_POP);
+      new SqlAvgAggFunction(SqlKind.VAR_POP);
 
   /**
    * <code>VAR_SAMP</code> aggregate function.
    */
   public static final SqlAggFunction VAR_SAMP =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.VAR_SAMP);
+      new SqlAvgAggFunction(SqlKind.VAR_SAMP);
 
   //-------------------------------------------------------------
   // WINDOW Aggregate Functions
@@ -1509,24 +1500,25 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    */
   public static final SqlAggFunction COLLECT =
       new SqlAggFunction("COLLECT",
-          SqlKind.OTHER_FUNCTION,
+          null,
+          SqlKind.COLLECT,
           ReturnTypes.TO_MULTISET,
           null,
           OperandTypes.ANY,
-          SqlFunctionCategory.SYSTEM) {
+          SqlFunctionCategory.SYSTEM, false, false) {
       };
 
   /**
    * The FUSION operator. Multiset aggregator function.
    */
   public static final SqlFunction FUSION =
-      new SqlFunction(
-          "FUSION",
-          SqlKind.OTHER_FUNCTION,
+      new SqlAggFunction("FUSION", null,
+          SqlKind.FUSION,
           ReturnTypes.ARG0,
           null,
           OperandTypes.MULTISET,
-          SqlFunctionCategory.SYSTEM);
+          SqlFunctionCategory.SYSTEM, false, false) {
+      };
 
   /**
    * The sequence next value function: <code>NEXT VALUE FOR sequence</code>

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
index 48e8ea4..5bf124d 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
@@ -39,6 +39,7 @@ public class SqlSumAggFunction extends SqlAggFunction {
 
   //~ Instance fields --------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   private final RelDataType type;
 
   //~ Constructors -----------------------------------------------------------
@@ -47,7 +48,7 @@ public class SqlSumAggFunction extends SqlAggFunction {
     super(
         "SUM",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.SUM,
         ReturnTypes.AGG_SUM,
         null,
         OperandTypes.NUMERIC,
@@ -63,6 +64,7 @@ public class SqlSumAggFunction extends SqlAggFunction {
     return ImmutableList.of(type);
   }
 
+  @Deprecated // to be removed before 2.0
   public RelDataType getType() {
     return type;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
index 5b673e2..e8f7844 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
@@ -42,7 +42,7 @@ public class SqlSumEmptyIsZeroAggFunction extends SqlAggFunction {
   SqlSumEmptyIsZeroAggFunction() {
     super("$SUM0",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.SUM0,
         ReturnTypes.AGG_SUM_EMPTY_IS_ZERO,
         null,
         OperandTypes.NUMERIC,


[6/6] calcite git commit: [CALCITE-1038] Obsolete AggregateConstantKeyRule

Posted by jh...@apache.org.
[CALCITE-1038] Obsolete AggregateConstantKeyRule

Move AggregateConstantKeyRule's functionality into
AggregateProjectPullUpConstantsRule and obsolete it.

Detect constants using metadata (RelMdPredicates). The input does not
need to be a Project, and constants do not need to be literals.


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

Branch: refs/heads/master
Commit: 904c73da60b9f9deec61ea34d89ada3462381f93
Parents: 8531ab1
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jan 4 10:35:43 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 6 02:37:31 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/plan/RelOptUtil.java     |   4 +-
 .../rel/rules/AggregateConstantKeyRule.java     | 130 ---------------
 .../AggregateProjectPullUpConstantsRule.java    | 163 +++++++++----------
 .../rel/rules/ReduceExpressionsRule.java        |  67 +++++---
 .../apache/calcite/test/RelOptRulesTest.java    |  15 +-
 .../org/apache/calcite/test/RelOptRulesTest.xml |  14 +-
 6 files changed, 135 insertions(+), 258 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/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 e658749..e444612 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -39,7 +39,6 @@ import org.apache.calcite.rel.logical.LogicalCalc;
 import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.rules.AggregateConstantKeyRule;
 import org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule;
 import org.apache.calcite.rel.rules.FilterMergeRule;
 import org.apache.calcite.rel.rules.MultiJoin;
@@ -1546,8 +1545,7 @@ public abstract class RelOptUtil {
   }
 
   public static void registerAbstractRels(RelOptPlanner planner) {
-    planner.addRule(AggregateProjectPullUpConstantsRule.INSTANCE);
-    planner.addRule(AggregateConstantKeyRule.INSTANCE);
+    planner.addRule(AggregateProjectPullUpConstantsRule.INSTANCE2);
     planner.addRule(PruneEmptyRules.UNION_INSTANCE);
     planner.addRule(PruneEmptyRules.PROJECT_INSTANCE);
     planner.addRule(PruneEmptyRules.FILTER_INSTANCE);

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/core/src/main/java/org/apache/calcite/rel/rules/AggregateConstantKeyRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateConstantKeyRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateConstantKeyRule.java
deleted file mode 100644
index ea31178..0000000
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateConstantKeyRule.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.rel.rules;
-
-import org.apache.calcite.plan.RelOptPredicateList;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.tools.RelBuilder;
-import org.apache.calcite.tools.RelBuilderFactory;
-import org.apache.calcite.util.ImmutableBitSet;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
-/**
- * Planner rule that removes constant keys from an
- * a {@link Aggregate}.
- *
- * <p>It never removes the last column, because {@code Aggregate([])} returns
- * 1 row even if its input is empty.
- */
-public class AggregateConstantKeyRule extends RelOptRule {
-  public static final AggregateConstantKeyRule INSTANCE =
-      new AggregateConstantKeyRule(RelFactories.LOGICAL_BUILDER,
-          "AggregateConstantKeyRule");
-
-  //~ Constructors -----------------------------------------------------------
-
-  /** Creates an AggregateConstantKeyRule. */
-  private AggregateConstantKeyRule(RelBuilderFactory relBuilderFactory,
-      String description) {
-    super(operand(Aggregate.class, null, Aggregate.IS_SIMPLE, any()),
-        relBuilderFactory, description);
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  public void onMatch(RelOptRuleCall call) {
-    final Aggregate aggregate = call.rel(0);
-    assert !aggregate.indicator : "predicate ensured no grouping sets";
-
-    final RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder();
-    final RelOptPredicateList predicates =
-        RelMetadataQuery.getPulledUpPredicates(aggregate.getInput());
-    if (predicates == null) {
-      return;
-    }
-    final ImmutableMap<RexNode, RexLiteral> constants =
-        ReduceExpressionsRule.predicateConstants(rexBuilder, predicates);
-    final NavigableMap<Integer, RexLiteral> map = new TreeMap<>();
-    for (int key : aggregate.getGroupSet()) {
-      final RexInputRef ref =
-          rexBuilder.makeInputRef(aggregate.getInput(), key);
-      if (constants.containsKey(ref)) {
-        map.put(key, constants.get(ref));
-      }
-    }
-
-    if (map.isEmpty()) {
-      return; // none of the keys are constant
-    }
-
-    if (map.size() == aggregate.getGroupCount()) {
-      if (map.size() == 1) {
-        // There is one key, and it is constant. We cannot remove it.
-        return;
-      }
-      map.remove(map.descendingKeySet().descendingIterator().next());
-    }
-
-    ImmutableBitSet newGroupSet = aggregate.getGroupSet();
-    for (int key : map.keySet()) {
-      newGroupSet = newGroupSet.clear(key);
-    }
-    final Aggregate newAggregate =
-        aggregate.copy(aggregate.getTraitSet(), aggregate.getInput(),
-            false, newGroupSet, ImmutableList.of(newGroupSet),
-            aggregate.getAggCallList());
-    final RelBuilder relBuilder = call.builder();
-    relBuilder.push(newAggregate);
-
-    final List<RexNode> projects = new ArrayList<>();
-    int offset = 0;
-    for (RelDataTypeField field : aggregate.getRowType().getFieldList()) {
-      RexNode node = null;
-      if (field.getIndex() < aggregate.getGroupCount()) {
-        node = map.get(aggregate.getGroupSet().nth(field.getIndex()));
-        if (node != null) {
-          node = relBuilder.getRexBuilder().makeCast(field.getType(), node, true);
-          node = relBuilder.alias(node, field.getName());
-          ++offset;
-        }
-      }
-      if (node == null) {
-        node = relBuilder.field(field.getIndex() - offset);
-      }
-      projects.add(node);
-    }
-    call.transformTo(relBuilder.project(projects).build());
-  }
-}
-
-// End AggregateConstantKeyRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
index 2b99cf6..a0ff130 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
@@ -16,70 +16,91 @@
  */
 package org.apache.calcite.rel.rules;
 
+import org.apache.calcite.plan.RelOptPredicateList;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Permutation;
-import org.apache.calcite.util.mapping.Mapping;
-import org.apache.calcite.util.mapping.MappingType;
+
+import com.google.common.collect.ImmutableMap;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 
 /**
- * Planner rule that removes constant expressions from the
- * group list of an {@link org.apache.calcite.rel.logical.LogicalAggregate}.
+ * Planner rule that removes constant keys from an
+ * {@link org.apache.calcite.rel.core.Aggregate}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
  *
- * <p><b>Effect of the rule</b></p>
+ * <p>This rules never removes the last column, because {@code Aggregate([])}
+ * returns 1 row even if its input is empty.
  *
  * <p>Since the transformed relational expression has to match the original
  * relational expression, the constants are placed in a projection above the
  * reduced aggregate. If those constants are not used, another rule will remove
  * them from the project.
- *
- * <p>LogicalAggregate needs its group columns to be on the prefix of its input
- * relational expression. Therefore, if a constant is not on the trailing edge
- * of the group list, removing it will leave a hole. In this case, the rule adds
- * a project before the aggregate to reorder the columns, and permutes them back
- * afterwards.
  */
 public class AggregateProjectPullUpConstantsRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
   /** The singleton. */
   public static final AggregateProjectPullUpConstantsRule INSTANCE =
-      new AggregateProjectPullUpConstantsRule();
+      new AggregateProjectPullUpConstantsRule(LogicalAggregate.class,
+          LogicalProject.class, RelFactories.LOGICAL_BUILDER,
+          "AggregateProjectPullUpConstantsRule");
+
+  /** More general instance that matches any relational expression. */
+  public static final AggregateProjectPullUpConstantsRule INSTANCE2 =
+      new AggregateProjectPullUpConstantsRule(LogicalAggregate.class,
+          RelNode.class, RelFactories.LOGICAL_BUILDER,
+          "AggregatePullUpConstantsRule");
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Private: use singleton
+   * Creates an AggregateProjectPullUpConstantsRule.
+   *
+   * @param aggregateClass Aggregate class
+   * @param inputClass Input class, such as {@link LogicalProject}
+   * @param relBuilderFactory Builder for relational expressions
+   * @param description Description, or null to guess description
    */
-  private AggregateProjectPullUpConstantsRule() {
+  public AggregateProjectPullUpConstantsRule(
+      Class<? extends Aggregate> aggregateClass,
+      Class<? extends RelNode> inputClass,
+      RelBuilderFactory relBuilderFactory, String description) {
     super(
-        operand(LogicalAggregate.class, null, Aggregate.IS_SIMPLE,
-            operand(LogicalProject.class, any())));
+        operand(aggregateClass, null, Aggregate.IS_SIMPLE,
+            operand(inputClass, any())),
+        relBuilderFactory, description);
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    final LogicalAggregate aggregate = call.rel(0);
-    final LogicalProject input = call.rel(1);
+    final Aggregate aggregate = call.rel(0);
+    final RelNode input = call.rel(1);
 
+    assert !aggregate.indicator : "predicate ensured no grouping sets";
     final int groupCount = aggregate.getGroupCount();
     if (groupCount == 1) {
       // No room for optimization since we cannot convert from non-empty
@@ -87,47 +108,49 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
       return;
     }
 
-    final RexProgram program =
-      RexProgram.create(input.getInput().getRowType(),
-          input.getProjects(),
-          null,
-          input.getRowType(),
-          input.getCluster().getRexBuilder());
-
-    final RelDataType childRowType = input.getRowType();
-    final List<Integer> constantList = new ArrayList<>();
-    final Map<Integer, RexNode> constants = new HashMap<>();
-    for (int i : aggregate.getGroupSet()) {
-      final RexLocalRef ref = program.getProjectList().get(i);
-      if (program.isConstant(ref)) {
-        constantList.add(i);
-        constants.put(
-            i,
-            program.gatherExpr(ref));
+    final RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder();
+    final RelOptPredicateList predicates =
+        RelMetadataQuery.getPulledUpPredicates(aggregate.getInput());
+    if (predicates == null) {
+      return;
+    }
+    final ImmutableMap<RexNode, RexNode> constants =
+        ReduceExpressionsRule.predicateConstants(RexNode.class, rexBuilder,
+            predicates);
+    final NavigableMap<Integer, RexNode> map = new TreeMap<>();
+    for (int key : aggregate.getGroupSet()) {
+      final RexInputRef ref =
+          rexBuilder.makeInputRef(aggregate.getInput(), key);
+      if (constants.containsKey(ref)) {
+        map.put(key, constants.get(ref));
       }
     }
 
     // None of the group expressions are constant. Nothing to do.
-    if (constantList.size() == 0) {
+    if (map.isEmpty()) {
       return;
     }
 
-    if (groupCount == constantList.size()) {
+    if (groupCount == map.size()) {
       // At least a single item in group by is required.
-      // Otherwise group by 1,2 might be altered to group by ()
+      // Otherwise "GROUP BY 1, 2" might be altered to "GROUP BY ()".
       // Removing of the first element is not optimal here,
       // however it will allow us to use fast path below (just trim
-      // groupCount)
-      constantList.remove(0);
+      // groupCount).
+      map.remove(map.navigableKeySet().first());
     }
 
-    final int newGroupCount = groupCount - constantList.size();
+    ImmutableBitSet newGroupSet = aggregate.getGroupSet();
+    for (int key : map.keySet()) {
+      newGroupSet = newGroupSet.clear(key);
+    }
+    final int newGroupCount = newGroupSet.cardinality();
 
     // If the constants are on the trailing edge of the group list, we just
     // reduce the group count.
     final RelBuilder relBuilder = call.builder();
     relBuilder.push(input);
-    if (constantList.get(0) == newGroupCount) {
+    if (map.navigableKeySet().first() == newGroupCount) {
       // Clone aggregate calls.
       final List<AggregateCall> newAggCalls = new ArrayList<>();
       for (AggregateCall aggCall : aggregate.getAggCallList()) {
@@ -136,13 +159,13 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
                 groupCount, newGroupCount));
       }
       relBuilder.aggregate(
-          relBuilder.groupKey(ImmutableBitSet.range(newGroupCount), false, null),
+          relBuilder.groupKey(newGroupSet, false, null),
           newAggCalls);
     } else {
       // Create the mapping from old field positions to new field
       // positions.
       final Permutation mapping =
-          new Permutation(childRowType.getFieldCount());
+          new Permutation(input.getRowType().getFieldCount());
       mapping.identity();
 
       // Ensure that the first positions in the mapping are for the new
@@ -150,18 +173,13 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
       for (int i = 0, groupOrdinal = 0, constOrdinal = newGroupCount;
           i < groupCount;
           ++i) {
-        if (i >= groupCount) {
-          mapping.set(i, i);
-        } else if (constants.containsKey(i)) {
+        if (map.containsKey(i)) {
           mapping.set(i, constOrdinal++);
         } else {
           mapping.set(i, groupOrdinal++);
         }
       }
 
-      // Create a projection to permute fields into these positions.
-      createProjection(relBuilder, mapping);
-
       // Adjust aggregate calls for new field positions.
       final List<AggregateCall> newAggCalls = new ArrayList<>();
       for (AggregateCall aggCall : aggregate.getAggCallList()) {
@@ -180,7 +198,7 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
 
       // Aggregate on projection.
       relBuilder.aggregate(
-          relBuilder.groupKey(ImmutableBitSet.range(newGroupCount), false, null),
+          relBuilder.groupKey(newGroupSet, false, null),
               newAggCalls);
     }
 
@@ -192,10 +210,10 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
       final int i = field.getIndex();
       if (i >= groupCount) {
         // Aggregate expressions' names and positions are unchanged.
-        expr = relBuilder.field(i - constantList.size());
-      } else if (constantList.contains(i)) {
+        expr = relBuilder.field(i - map.size());
+      } else if (map.containsKey(i)) {
         // Re-generate the constant expression in the project.
-        expr = constants.get(i);
+        expr = map.get(i);
       } else {
         // Project the aggregation expression, in its original
         // position.
@@ -208,33 +226,6 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
     call.transformTo(relBuilder.build());
   }
 
-  /**
-   * Creates a projection which permutes the fields of a given relational
-   * expression.
-   *
-   * <p>For example, given a relational expression [A, B, C, D] and a mapping
-   * [2:1, 3:0], returns a projection [$3 AS C, $2 AS B].
-   *
-   * @param relBuilder Relational expression builder
-   * @param mapping Mapping to apply to source columns
-   */
-  private static RelBuilder createProjection(RelBuilder relBuilder,
-      Mapping mapping) {
-    // Every target has precisely one source; every source has at most
-    // one target.
-    assert mapping.getMappingType().isA(MappingType.INVERSE_SURJECTION);
-    final RelDataType childRowType = relBuilder.peek().getRowType();
-    assert mapping.getSourceCount() == childRowType.getFieldCount();
-    final List<Pair<RexNode, String>> projects = new ArrayList<>();
-    for (int target = 0; target < mapping.getTargetCount(); ++target) {
-      int source = mapping.getSource(target);
-      projects.add(
-          Pair.<RexNode, String>of(
-              relBuilder.field(source),
-              childRowType.getFieldList().get(source).getName()));
-    }
-    return relBuilder.project(Pair.left(projects), Pair.right(projects));
-  }
 }
 
 // End AggregateProjectPullUpConstantsRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
index 8be202e..17d5cb0 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
@@ -396,7 +396,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     List<Boolean> addCasts = Lists.newArrayList();
     final List<RexNode> removableCasts = Lists.newArrayList();
     final ImmutableMap<RexNode, RexLiteral> constants =
-        predicateConstants(rexBuilder, predicates);
+        predicateConstants(RexLiteral.class, rexBuilder, predicates);
     findReducibleExps(rel.getCluster().getTypeFactory(), expList, constants,
         constExps, addCasts, removableCasts);
     if (constExps.isEmpty() && removableCasts.isEmpty()) {
@@ -495,7 +495,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
    * @param removableCasts returns the list of cast expressions where the cast
    */
   protected static void findReducibleExps(RelDataTypeFactory typeFactory,
-      List<RexNode> exps, ImmutableMap<RexNode, RexLiteral> constants,
+      List<RexNode> exps, ImmutableMap<RexNode, ? extends RexNode> constants,
       List<RexNode> constExps, List<Boolean> addCasts,
       List<RexNode> removableCasts) {
     ReducibleExprLocator gardener =
@@ -507,7 +507,19 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     assert constExps.size() == addCasts.size();
   }
 
-  protected static ImmutableMap<RexNode, RexLiteral> predicateConstants(
+  /** Creates a map containing each (e, constant) pair that occurs within
+   * a predicate list.
+   *
+   * @param clazz Class of expression that is considered constant
+   * @param rexBuilder Rex builder
+   * @param predicates Predicate list
+   * @param <C> what to consider a constant: {@link RexLiteral} to use a narrow
+   *           definition of constant, or {@link RexNode} to use
+   *           {@link RexUtil#isConstant(RexNode)}
+   * @return Map from values to constants
+   */
+  protected static <C extends RexNode> ImmutableMap<RexNode, C>
+  predicateConstants(Class<C> clazz,
       RexBuilder rexBuilder, RelOptPredicateList predicates) {
     // We cannot use an ImmutableMap.Builder here. If there are multiple entries
     // with the same key (e.g. "WHERE deptno = 1 AND deptno = 2"), it doesn't
@@ -518,14 +530,14 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     // (e.g. "WHERE deptno = 1 AND deptno = 1")
     // (3) It will return false if there are inconsistent constraints (e.g.
     // "WHERE deptno = 1 AND deptno = 2")
-    final Map<RexNode, RexLiteral> map = new HashMap<>();
+    final Map<RexNode, C> map = new HashMap<>();
     final Set<RexNode> excludeSet = new HashSet<>();
     for (RexNode predicate : predicates.pulledUpPredicates) {
-      gatherConstraints(predicate, map, excludeSet, rexBuilder);
+      gatherConstraints(clazz, predicate, map, excludeSet, rexBuilder);
     }
-    final ImmutableMap.Builder<RexNode, RexLiteral> builder =
+    final ImmutableMap.Builder<RexNode, C> builder =
         ImmutableMap.builder();
-    for (Map.Entry<RexNode, RexLiteral> entry : map.entrySet()) {
+    for (Map.Entry<RexNode, C> entry : map.entrySet()) {
       RexNode rexNode = entry.getKey();
       if (!overlap(rexNode, excludeSet)) {
         builder.put(rexNode, entry.getValue());
@@ -559,8 +571,8 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     }
   }
 
-  private static void gatherConstraints(RexNode predicate,
-      Map<RexNode, RexLiteral> map, Set<RexNode> excludeSet,
+  private static <C extends RexNode> void gatherConstraints(Class<C> clazz,
+      RexNode predicate, Map<RexNode, C> map, Set<RexNode> excludeSet,
       RexBuilder rexBuilder) {
     if (predicate.getKind() != SqlKind.EQUALS) {
       decompose(excludeSet, predicate);
@@ -576,21 +588,24 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     final RexNode right = operands.get(1);
     // note that literals are immutable too and they can only be compared through
     // values.
-    if (right instanceof RexLiteral) {
-      foo(left, (RexLiteral) right, map, excludeSet, rexBuilder);
-    }
-    if (left instanceof RexLiteral) {
-      foo(right, (RexLiteral) left, map, excludeSet, rexBuilder);
-    }
+    gatherConstraint(clazz, left, right, map, excludeSet, rexBuilder);
+    gatherConstraint(clazz, right, left, map, excludeSet, rexBuilder);
   }
 
-  private static void foo(RexNode left, RexLiteral right,
-      Map<RexNode, RexLiteral> map, Set<RexNode> excludeSet,
+  private static <C extends RexNode> void gatherConstraint(Class<C> clazz,
+      RexNode left, RexNode right, Map<RexNode, C> map, Set<RexNode> excludeSet,
       RexBuilder rexBuilder) {
+    if (!clazz.isInstance(right)) {
+      return;
+    }
+    if (!RexUtil.isConstant(right)) {
+      return;
+    }
+    C constant = clazz.cast(right);
     if (excludeSet.contains(left)) {
       return;
     }
-    final RexLiteral existedValue = map.get(left);
+    final C existedValue = map.get(left);
     if (existedValue == null) {
       switch (left.getKind()) {
       case CAST:
@@ -599,16 +614,19 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
         final RexNode operand = ((RexCall) left).getOperands().get(0);
         if (canAssignFrom(left.getType(), operand.getType())) {
           final RexNode castRight =
-              rexBuilder.makeCast(operand.getType(), right);
+              rexBuilder.makeCast(operand.getType(), constant);
           if (castRight instanceof RexLiteral) {
             left = operand;
-            right = (RexLiteral) castRight;
+            constant = clazz.cast(castRight);
           }
         }
       }
-      map.put(left, right);
+      map.put(left, constant);
     } else {
-      if (!existedValue.getValue().equals(right.getValue())) {
+      if (existedValue instanceof RexLiteral
+          && constant instanceof RexLiteral
+          && !((RexLiteral) existedValue).getValue()
+              .equals(((RexLiteral) constant).getValue())) {
         // we found conflicting values, e.g. left = 10 and left = 20
         map.remove(left);
         excludeSet.add(left);
@@ -770,7 +788,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
 
     private final List<Constancy> stack;
 
-    private final ImmutableMap<RexNode, RexLiteral> constants;
+    private final ImmutableMap<RexNode, ? extends RexNode> constants;
 
     private final List<RexNode> constExprs;
 
@@ -781,7 +799,8 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     private final List<SqlOperator> parentCallTypeStack;
 
     ReducibleExprLocator(RelDataTypeFactory typeFactory,
-        ImmutableMap<RexNode, RexLiteral> constants, List<RexNode> constExprs,
+        ImmutableMap<RexNode, ? extends RexNode> constants,
+        List<RexNode> constExprs,
         List<Boolean> addCasts, List<RexNode> removableCasts) {
       // go deep
       super(true);

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 4061b0d..92bae3e 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -36,7 +36,6 @@ import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
 import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
-import org.apache.calcite.rel.rules.AggregateConstantKeyRule;
 import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
 import org.apache.calcite.rel.rules.AggregateFilterTransposeRule;
 import org.apache.calcite.rel.rules.AggregateJoinTransposeRule;
@@ -2027,7 +2026,7 @@ public class RelOptRulesTest extends RelOptTestBase {
    * Planner rule that removes Aggregate keys that are constant</a>. */
   @Test public void testAggregateConstantKeyRule() {
     final HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(AggregateConstantKeyRule.INSTANCE)
+        .addRuleInstance(AggregateProjectPullUpConstantsRule.INSTANCE2)
         .build();
     final String sql = "select count(*) as c\n"
         + "from sales.emp\n"
@@ -2036,11 +2035,11 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkPlanning(new HepPlanner(program), sql);
   }
 
-  /** Tests {@link AggregateConstantKeyRule} where reduction is not possible
-   * because "deptno" is the only key. */
+  /** Tests {@link AggregateProjectPullUpConstantsRule} where reduction is not
+   * possible because "deptno" is the only key. */
   @Test public void testAggregateConstantKeyRule2() {
     final HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(AggregateConstantKeyRule.INSTANCE)
+        .addRuleInstance(AggregateProjectPullUpConstantsRule.INSTANCE2)
         .build();
     final String sql = "select count(*) as c\n"
         + "from sales.emp\n"
@@ -2049,11 +2048,11 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkPlanUnchanged(new HepPlanner(program), sql);
   }
 
-  /** Tests {@link AggregateConstantKeyRule} where both keys are constants but
-   * only one can be removed. */
+  /** Tests {@link AggregateProjectPullUpConstantsRule} where both keys are
+   * constants but only one can be removed. */
   @Test public void testAggregateConstantKeyRule3() {
     final HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(AggregateConstantKeyRule.INSTANCE)
+        .addRuleInstance(AggregateProjectPullUpConstantsRule.INSTANCE2)
         .build();
     final String sql = "select job\n"
         + "from sales.emp\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index 38edf69..c9c9342 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -1795,8 +1795,8 @@ LogicalProject(DEPTNO=[$0], EXPR$1=[$4])
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(DEPTNO=[$0], EXPR$1=[$2])
-  LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($4)])
-    LogicalProject(DEPTNO=[$7], DEPTNO42=[+($7, 42)], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], MGR=[$3])
+  LogicalAggregate(group=[{0, 3}], EXPR$1=[MAX($4)])
+    LogicalProject(DEPTNO=[$7], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)], MGR=[$3])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -1818,8 +1818,8 @@ LogicalProject(DEPTNO=[$1], EXPR$1=[$2])
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalAggregate(group=[{0}], EXPR$1=[MAX($2)])
-  LogicalProject(DEPTNO=[$7], FOUR=[4], MGR=[$3])
+LogicalAggregate(group=[{1}], EXPR$1=[MAX($2)])
+  LogicalProject(FOUR=[4], DEPTNO=[$7], MGR=[$3])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -1841,8 +1841,8 @@ LogicalProject(DEPTNO=[$1], EXPR$1=[$2])
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalAggregate(group=[{0}], EXPR$1=[MAX($2)])
-  LogicalProject(DEPTNO=[$7], $f0=[+(42, 24)], MGR=[$3])
+LogicalAggregate(group=[{1}], EXPR$1=[MAX($2)])
+  LogicalProject($f0=[+(42, 24)], DEPTNO=[$7], MGR=[$3])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -4311,7 +4311,7 @@ LogicalProject(JOB=[$1])
             <![CDATA[
 LogicalProject(JOB=[$1])
   LogicalFilter(condition=[>($2, 3)])
-    LogicalProject(SAL=[$0], JOB=['Clerk'], $f1=[$1])
+    LogicalProject(SAL=[$0], JOB=['Clerk'], $f2=[$1])
       LogicalAggregate(group=[{0}], agg#0=[COUNT()])
         LogicalProject(SAL=[$5], JOB=[$2])
           LogicalFilter(condition=[AND(IS NULL($5), =($2, 'Clerk'))])


[4/6] calcite git commit: [CALCITE-1041] User-defined function that returns DATE or TIMESTAMP value

Posted by jh...@apache.org.
[CALCITE-1041] User-defined function that returns DATE or TIMESTAMP value

Rationalize code that translates to an from internal representation (e.g. DATE stored as int).

Fix an bug with overloaded UDFs, and a bug with UDF args of type SMALLINT (short).

Deprecate Utilities.equal.

Add methods to in BuiltInMethod.


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

Branch: refs/heads/master
Commit: d4bbf58d6e5f4f67ebae412b5a1aae23769eccb7
Parents: efb668b
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jan 5 15:15:51 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 6 02:37:31 2016 -0800

----------------------------------------------------------------------
 .../calcite/adapter/enumerable/EnumUtils.java   | 118 ++++++++++++++++++-
 .../enumerable/EnumerableRelImplementor.java    |  19 +--
 .../adapter/enumerable/PhysTypeImpl.java        |  15 +--
 .../ReflectiveCallNotNullImplementor.java       |  36 +-----
 .../calcite/adapter/enumerable/RexImpTable.java |   7 +-
 .../adapter/enumerable/RexToLixTranslator.java  |  39 ++++--
 .../enumerable/StrictAggImplementor.java        |  22 ++--
 .../calcite/prepare/CalciteCatalogReader.java   |  48 +++++---
 .../org/apache/calcite/runtime/FlatLists.java   |  11 +-
 .../apache/calcite/runtime/SqlFunctions.java    |  37 ++++--
 .../org/apache/calcite/runtime/Utilities.java   |   2 +
 .../java/org/apache/calcite/sql/SqlUtil.java    |   2 +-
 .../sql/type/SqlTypeExplicitPrecedenceList.java |   1 +
 .../apache/calcite/sql/type/SqlTypeUtil.java    |   4 +
 .../org/apache/calcite/util/BuiltInMethod.java  |  16 ++-
 .../org/apache/calcite/test/CalciteAssert.java  |   3 +-
 .../java/org/apache/calcite/test/UdfTest.java   |  62 ++++++++++
 .../java/org/apache/calcite/util/Smalls.java    |  35 ++++--
 18 files changed, 357 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
index 17e0bf5..60408c6 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
@@ -20,6 +20,7 @@ import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.function.Function2;
 import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.ConstantUntypedNull;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.MethodDeclaration;
@@ -29,8 +30,11 @@ import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
 
+import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
@@ -44,6 +48,14 @@ import java.util.List;
  * style.
  */
 public class EnumUtils {
+
+  private static final Function<RexNode, Type> REX_TO_INTERNAL_TYPE =
+      new Function<RexNode, Type>() {
+        public Type apply(RexNode node) {
+          return toInternal(node.getType());
+        }
+      };
+
   private EnumUtils() {}
 
   static final boolean BRIDGE_METHODS = true;
@@ -54,7 +66,8 @@ public class EnumUtils {
   static final List<Expression> NO_EXPRS =
       ImmutableList.of();
 
-  public static final String[] LEFT_RIGHT = {"left", "right"};
+  public static final List<String> LEFT_RIGHT =
+      ImmutableList.of("left", "right");
 
   /** Declares a method that overrides another method. */
   public static MethodDeclaration overridingMethodDecl(Method method,
@@ -117,12 +130,10 @@ public class EnumUtils {
   static Expression joinSelector(JoinRelType joinType, PhysType physType,
       List<PhysType> inputPhysTypes) {
     // A parameter for each input.
-    final List<ParameterExpression> parameters =
-        new ArrayList<ParameterExpression>();
+    final List<ParameterExpression> parameters = new ArrayList<>();
 
     // Generate all fields.
-    final List<Expression> expressions =
-        new ArrayList<Expression>();
+    final List<Expression> expressions = new ArrayList<>();
     final int outputFieldCount = physType.getRowType().getFieldCount();
     for (Ord<PhysType> ord : Ord.zip(inputPhysTypes)) {
       final PhysType inputPhysType =
@@ -132,7 +143,7 @@ public class EnumUtils {
       // Function<T> always operates on boxed arguments
       final ParameterExpression parameter =
           Expressions.parameter(Primitive.box(inputPhysType.getJavaRowType()),
-              EnumUtils.LEFT_RIGHT[ord.i]);
+              EnumUtils.LEFT_RIGHT.get(ord.i));
       parameters.add(parameter);
       if (expressions.size() == outputFieldCount) {
         // For instance, if semi-join needs to return just the left inputs
@@ -158,6 +169,101 @@ public class EnumUtils {
         physType.record(expressions),
         parameters);
   }
+
+  /** Converts from internal representation to JDBC representation used by
+   * arguments of user-defined functions. For example, converts date values from
+   * {@code int} to {@link java.sql.Date}. */
+  static Expression fromInternal(Expression e, Class<?> targetType) {
+    if (e == ConstantUntypedNull.INSTANCE) {
+      return e;
+    }
+    if (!(e.getType() instanceof Class)) {
+      return e;
+    }
+    if (targetType.isAssignableFrom((Class) e.getType())) {
+      return e;
+    }
+    if (targetType == java.sql.Date.class) {
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_DATE.method, e);
+    }
+    if (targetType == java.sql.Time.class) {
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_TIME.method, e);
+    }
+    if (targetType == java.sql.Timestamp.class) {
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_TIMESTAMP.method, e);
+    }
+    if (Primitive.is(e.type)
+        && Primitive.isBox(targetType)) {
+      // E.g. e is "int", target is "Long", generate "(long) e".
+      return Expressions.convert_(e,
+          Primitive.ofBox(targetType).primitiveClass);
+    }
+    return e;
+  }
+
+  static List<Expression> fromInternal(Class<?>[] targetTypes,
+      List<Expression> expressions) {
+    final List<Expression> list = new ArrayList<>();
+    for (int i = 0; i < expressions.size(); i++) {
+      list.add(fromInternal(expressions.get(i), targetTypes[i]));
+    }
+    return list;
+  }
+
+  static Type fromInternal(Type type) {
+    if (type == java.sql.Date.class || type == java.sql.Time.class) {
+      return int.class;
+    }
+    if (type == java.sql.Timestamp.class) {
+      return long.class;
+    }
+    return type;
+  }
+
+  static Type toInternal(RelDataType type) {
+    switch (type.getSqlTypeName()) {
+    case DATE:
+    case TIME:
+      return type.isNullable() ? Integer.class : int.class;
+    case TIMESTAMP:
+      return type.isNullable() ? Long.class : long.class;
+    default:
+      return null; // we don't care; use the default storage type
+    }
+  }
+
+  static List<Type> internalTypes(List<? extends RexNode> operandList) {
+    return Lists.transform(operandList, REX_TO_INTERNAL_TYPE);
+  }
+
+  static Expression enforce(final Type storageType,
+      final Expression e) {
+    if (storageType != null && e.type != storageType) {
+      if (e.type == java.sql.Date.class) {
+        if (storageType == int.class) {
+          return Expressions.call(BuiltInMethod.DATE_TO_INT.method, e);
+        }
+        if (storageType == Integer.class) {
+          return Expressions.call(BuiltInMethod.DATE_TO_INT_OPTIONAL.method, e);
+        }
+      } else if (e.type == java.sql.Time.class) {
+        if (storageType == int.class) {
+          return Expressions.call(BuiltInMethod.TIME_TO_INT.method, e);
+        }
+        if (storageType == Integer.class) {
+          return Expressions.call(BuiltInMethod.TIME_TO_INT_OPTIONAL.method, e);
+        }
+      } else if (e.type == java.sql.Timestamp.class) {
+        if (storageType == long.class) {
+          return Expressions.call(BuiltInMethod.TIMESTAMP_TO_LONG.method, e);
+        }
+        if (storageType == Long.class) {
+          return Expressions.call(BuiltInMethod.TIMESTAMP_TO_LONG_OPTIONAL.method, e);
+        }
+      }
+    }
+    return e;
+  }
 }
 
 // End EnumUtils.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
index 1c446a3..bd77f2b 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
@@ -39,7 +39,6 @@ import org.apache.calcite.linq4j.tree.Types;
 import org.apache.calcite.linq4j.tree.Visitor;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.runtime.Bindable;
-import org.apache.calcite.runtime.Utilities;
 import org.apache.calcite.util.BuiltInMethod;
 
 import com.google.common.base.Function;
@@ -49,6 +48,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
 
 import java.io.Serializable;
+import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Type;
@@ -239,9 +239,7 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
               ? Expressions.equal(
                   Expressions.field(thisParameter, field.getName()),
                   Expressions.field(thatParameter, field.getName()))
-              : Expressions.call(
-                  Utilities.class,
-                  "equal",
+              : Expressions.call(BuiltInMethod.OBJECTS_EQUAL.method,
                   Expressions.field(thisParameter, field.getName()),
                   Expressions.field(thatParameter, field.getName())));
     }
@@ -270,13 +268,14 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
     blockBuilder3.add(
         Expressions.declare(0, hParameter, constantZero));
     for (Types.RecordField field : type.getRecordFields()) {
+      final Method method = BuiltInMethod.HASH.method;
       blockBuilder3.add(
           Expressions.statement(
               Expressions.assign(
                   hParameter,
                   Expressions.call(
-                      Utilities.class,
-                      "hash",
+                      method.getDeclaringClass(),
+                      method.getName(),
                       ImmutableList.of(
                           hParameter,
                           Expressions.field(thisParameter, field))))));
@@ -312,9 +311,11 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
     for (Types.RecordField field : type.getRecordFields()) {
       MethodCallExpression compareCall;
       try {
-        compareCall = Expressions.call(
-            Utilities.class,
-            field.nullable() ? "compareNullsLast" : "compare",
+        final Method method = (field.nullable()
+            ? BuiltInMethod.COMPARE_NULLS_LAST
+            : BuiltInMethod.COMPARE).method;
+        compareCall = Expressions.call(method.getDeclaringClass(),
+            method.getName(),
             Expressions.field(thisParameter, field),
             Expressions.field(thatParameter, field));
       } catch (RuntimeException e) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
index af42c16..918833e 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
@@ -40,6 +40,7 @@ import org.apache.calcite.util.Util;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
 import java.util.AbstractList;
@@ -301,17 +302,17 @@ public class PhysTypeImpl implements PhysType {
       final boolean descending =
           collation.getDirection()
               == RelFieldCollation.Direction.DESCENDING;
+      final Method method = (fieldNullable(index)
+          ? (nullsFirst ^ descending
+              ? BuiltInMethod.COMPARE_NULLS_FIRST
+              : BuiltInMethod.COMPARE_NULLS_LAST)
+          : BuiltInMethod.COMPARE).method;
       body.add(
           Expressions.statement(
               Expressions.assign(
                   parameterC,
-                  Expressions.call(
-                      Utilities.class,
-                      fieldNullable(index)
-                          ? (nullsFirst ^ descending
-                              ? "compareNullsFirst"
-                              : "compareNullsLast")
-                          : "compare",
+                  Expressions.call(method.getDeclaringClass(),
+                      method.getName(),
                       arg0,
                       arg1))));
       body.add(

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
index 6414e03..ee2ad1f 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
@@ -16,16 +16,13 @@
  */
 package org.apache.calcite.adapter.enumerable;
 
-import org.apache.calcite.linq4j.tree.ConstantUntypedNull;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.NewExpression;
 import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.runtime.SqlFunctions;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
-import java.util.ArrayList;
 import java.util.List;
 
 /**
@@ -49,7 +46,8 @@ public class ReflectiveCallNotNullImplementor implements NotNullImplementor {
 
   public Expression implement(RexToLixTranslator translator,
       RexCall call, List<Expression> translatedOperands) {
-    translatedOperands = fromInternal(translatedOperands);
+    translatedOperands =
+        EnumUtils.fromInternal(method.getParameterTypes(), translatedOperands);
     if ((method.getModifiers() & Modifier.STATIC) != 0) {
       return Expressions.call(method, translatedOperands);
     } else {
@@ -61,36 +59,6 @@ public class ReflectiveCallNotNullImplementor implements NotNullImplementor {
     }
   }
 
-  protected List<Expression> fromInternal(List<Expression> expressions) {
-    final List<Expression> list = new ArrayList<>();
-    final Class[] types = method.getParameterTypes();
-    for (int i = 0; i < expressions.size(); i++) {
-      list.add(fromInternal(expressions.get(i), types[i]));
-    }
-    return list;
-  }
-
-  protected Expression fromInternal(Expression e, Class<?> targetType) {
-    if (e == ConstantUntypedNull.INSTANCE) {
-      return e;
-    }
-    if (!(e.getType() instanceof Class)) {
-      return e;
-    }
-    if (targetType.isAssignableFrom((Class) e.getType())) {
-      return e;
-    }
-    if (targetType == java.sql.Date.class) {
-      return Expressions.call(SqlFunctions.class, "internalToDate", e);
-    }
-    if (targetType == java.sql.Time.class) {
-      return Expressions.call(SqlFunctions.class, "internalToTime", e);
-    }
-    if (targetType == java.sql.Timestamp.class) {
-      return Expressions.call(SqlFunctions.class, "internalToTimestamp", e);
-    }
-    return e;
-  }
 }
 
 // End ReflectiveCallNotNullImplementor.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index 32c043c..20872b6 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -1042,9 +1042,12 @@ public class RexImpTable {
       Expression acc = add.accumulator().get(0);
       Expression arg = add.arguments().get(0);
       SqlAggFunction aggregation = info.aggregation();
+      final Method method = (aggregation == MIN
+          ? BuiltInMethod.LESSER
+          : BuiltInMethod.GREATER).method;
       Expression next = Expressions.call(
-          SqlFunctions.class,
-          aggregation == MIN ? "lesser" : "greater",
+          method.getDeclaringClass(),
+          method.getName(),
           acc,
           Expressions.unbox(arg));
       accAdvance(add, acc, next);

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
index eed3d9d..2af6b46 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
@@ -181,7 +181,7 @@ public class RexToLixTranslator {
     List<Type> storageTypes = null;
     if (outputPhysType != null) {
       final RelDataType rowType = outputPhysType.getRowType();
-      storageTypes = new ArrayList<Type>(rowType.getFieldCount());
+      storageTypes = new ArrayList<>(rowType.getFieldCount());
       for (int i = 0; i < rowType.getFieldCount(); i++) {
         storageTypes.add(outputPhysType.getJavaFieldType(i));
       }
@@ -217,6 +217,7 @@ public class RexToLixTranslator {
   Expression translate(RexNode expr, RexImpTable.NullAs nullAs,
       Type storageType) {
     Expression expression = translate0(expr, nullAs, storageType);
+    expression = EnumUtils.enforce(storageType, expression);
     assert expression != null;
     return list.append("v", expression);
   }
@@ -500,9 +501,7 @@ public class RexToLixTranslator {
       }
       InputGetter getter =
           correlates.apply(((RexCorrelVariable) target).getName());
-      Expression res =
-          getter.field(list, fieldAccess.getField().getIndex(), storageType);
-      return res;
+      return getter.field(list, fieldAccess.getField().getIndex(), storageType);
     default:
       if (expr instanceof RexCall) {
         return translateCall((RexCall) expr, nullAs);
@@ -643,9 +642,17 @@ public class RexToLixTranslator {
   public List<Expression> translateList(
       List<RexNode> operandList,
       RexImpTable.NullAs nullAs) {
-    final List<Expression> list = new ArrayList<Expression>();
-    for (RexNode rex : operandList) {
-      list.add(translate(rex, nullAs));
+    return translateList(operandList, nullAs,
+        EnumUtils.internalTypes(operandList));
+  }
+
+  public List<Expression> translateList(
+      List<RexNode> operandList,
+      RexImpTable.NullAs nullAs,
+      List<? extends Type> storageTypes) {
+    final List<Expression> list = new ArrayList<>();
+    for (Pair<RexNode, ? extends Type> e : Pair.zip(operandList, storageTypes)) {
+      list.add(translate(e.left, nullAs, e.right));
     }
     return list;
   }
@@ -663,7 +670,7 @@ public class RexToLixTranslator {
    * @return translated expressions
    */
   public List<Expression> translateList(List<? extends RexNode> operandList) {
-    return translateList(operandList, null);
+    return translateList(operandList, EnumUtils.internalTypes(operandList));
   }
 
   /**
@@ -682,7 +689,7 @@ public class RexToLixTranslator {
    */
   public List<Expression> translateList(List<? extends RexNode> operandList,
       List<? extends Type> storageTypes) {
-    final List<Expression> list = new ArrayList<Expression>(operandList.size());
+    final List<Expression> list = new ArrayList<>(operandList.size());
 
     for (int i = 0; i < operandList.size(); i++) {
       RexNode rex = operandList.get(i);
@@ -812,6 +819,19 @@ public class RexToLixTranslator {
         }
       }
       return Expressions.box(operand, toBox);
+    } else if (toType == java.sql.Date.class) {
+      // E.g. from "int" or "Integer" to "java.sql.Date",
+      // generate "SqlFunctions.internalToDate".
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_DATE.method, operand);
+    } else if (toType == java.sql.Time.class) {
+      // E.g. from "int" or "Integer" to "java.sql.Time",
+      // generate "SqlFunctions.internalToTime".
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_TIME.method, operand);
+    } else if (toType == java.sql.Timestamp.class) {
+      // E.g. from "long" or "Long" to "java.sql.Timestamp",
+      // generate "SqlFunctions.internalToTimestamp".
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_TIMESTAMP.method,
+          operand);
     } else if (toType == BigDecimal.class) {
       if (fromBox != null) {
         // E.g. from "Integer" to "BigDecimal".
@@ -1048,6 +1068,7 @@ public class RexToLixTranslator {
    * it is not null. It is easier to throw (and caller will always handle)
    * than to check exhaustively beforehand. */
   static class AlwaysNull extends ControlFlowException {
+    @SuppressWarnings("ThrowableInstanceNeverThrown")
     public static final AlwaysNull INSTANCE = new AlwaysNull();
 
     private AlwaysNull() {}

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
index 8b463e8..a34813c 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
@@ -65,13 +65,7 @@ public abstract class StrictAggImplementor implements AggImplementor {
     if (!needTrackEmptySet) {
       return subState;
     }
-    boolean hasNullableArgs = false;
-    for (RelDataType type : info.parameterRelTypes()) {
-      if (type.isNullable()) {
-        hasNullableArgs = true;
-        break;
-      }
-    }
+    final boolean hasNullableArgs = anyNullable(info.parameterRelTypes());
     trackNullsPerRow = !(info instanceof WinAggContext) || hasNullableArgs;
 
     List<Type> res = new ArrayList<>(subState.size() + 1);
@@ -80,8 +74,20 @@ public abstract class StrictAggImplementor implements AggImplementor {
     return res;
   }
 
+  private boolean anyNullable(List<? extends RelDataType> types) {
+    for (RelDataType type : types) {
+      if (type.isNullable()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   public List<Type> getNotNullState(AggContext info) {
-    return Collections.singletonList(Primitive.unbox(info.returnType()));
+    Type type = info.returnType();
+    type = EnumUtils.fromInternal(type);
+    type = Primitive.unbox(type);
+    return Collections.singletonList(type);
   }
 
   public final void implementReset(AggContext info, AggResetContext reset) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/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 e3211ea..68b297e 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -27,18 +27,19 @@ import org.apache.calcite.schema.AggregateFunction;
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.FunctionParameter;
 import org.apache.calcite.schema.ScalarFunction;
-import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableFunction;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlSyntax;
 import org.apache.calcite.sql.type.FamilyOperandTypeChecker;
 import org.apache.calcite.sql.type.InferTypes;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlMoniker;
@@ -263,14 +264,11 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
         OperandTypes.family(typeFamilies, optional);
     final List<RelDataType> paramTypes = toSql(argTypes);
     if (function instanceof ScalarFunction) {
-      return new SqlUserDefinedFunction(name,
-          ReturnTypes.explicit(Schemas.proto((ScalarFunction) function)),
+      return new SqlUserDefinedFunction(name, infer((ScalarFunction) function),
           InferTypes.explicit(argTypes), typeChecker, paramTypes, function);
     } else if (function instanceof AggregateFunction) {
-      final RelDataType returnType =
-          ((AggregateFunction) function).getReturnType(typeFactory);
       return new SqlUserDefinedAggFunction(name,
-          ReturnTypes.explicit(returnType), InferTypes.explicit(argTypes),
+          infer((AggregateFunction) function), InferTypes.explicit(argTypes),
           typeChecker, (AggregateFunction) function);
     } else if (function instanceof TableMacro) {
       return new SqlUserDefinedTableMacro(name, ReturnTypes.CURSOR,
@@ -285,21 +283,43 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
     }
   }
 
+  private SqlReturnTypeInference infer(final ScalarFunction function) {
+    return new SqlReturnTypeInference() {
+      public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+        final RelDataType type = function.getReturnType(typeFactory);
+        return toSql(type);
+      }
+    };
+  }
+
+  private SqlReturnTypeInference infer(final AggregateFunction function) {
+    return new SqlReturnTypeInference() {
+      public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+        final RelDataType type = function.getReturnType(typeFactory);
+        return toSql(type);
+      }
+    };
+  }
+
   private List<RelDataType> toSql(List<RelDataType> types) {
     return Lists.transform(types,
         new com.google.common.base.Function<RelDataType, RelDataType>() {
-          public RelDataType apply(RelDataType input) {
-            if (input instanceof RelDataTypeFactoryImpl.JavaType
-                && ((RelDataTypeFactoryImpl.JavaType) input).getJavaClass()
-                == Object.class) {
-              return typeFactory.createTypeWithNullability(
-                  typeFactory.createSqlType(SqlTypeName.ANY), true);
-            }
-            return typeFactory.toSql(input);
+          public RelDataType apply(RelDataType type) {
+            return toSql(type);
           }
         });
   }
 
+  private RelDataType toSql(RelDataType type) {
+    if (type instanceof RelDataTypeFactoryImpl.JavaType
+        && ((RelDataTypeFactoryImpl.JavaType) type).getJavaClass()
+        == Object.class) {
+      return typeFactory.createTypeWithNullability(
+          typeFactory.createSqlType(SqlTypeName.ANY), true);
+    }
+    return typeFactory.toSql(type);
+  }
+
   public List<SqlOperator> getOperatorList() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/FlatLists.java b/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
index 4a001cf..f3cfae4 100644
--- a/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
+++ b/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
@@ -23,6 +23,7 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.ListIterator;
+import java.util.Objects;
 import java.util.RandomAccess;
 
 /**
@@ -261,8 +262,8 @@ public class FlatLists {
       }
       if (o instanceof Flat2List) {
         Flat2List that = (Flat2List) o;
-        return Utilities.equal(this.t0, that.t0)
-            && Utilities.equal(this.t1, that.t1);
+        return Objects.equals(this.t0, that.t0)
+            && Objects.equals(this.t1, that.t1);
       }
       return Arrays.asList(t0, t1).equals(o);
     }
@@ -386,9 +387,9 @@ public class FlatLists {
       }
       if (o instanceof Flat3List) {
         Flat3List that = (Flat3List) o;
-        return Utilities.equal(this.t0, that.t0)
-            && Utilities.equal(this.t1, that.t1)
-            && Utilities.equal(this.t2, that.t2);
+        return Objects.equals(this.t0, that.t0)
+            && Objects.equals(this.t1, that.t1)
+            && Objects.equals(this.t2, that.t2);
       }
       return o.equals(this);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index 22b0aaa..8abdb97 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -1040,6 +1040,10 @@ public class SqlFunctions {
         : (Short) cannotConvert(o, short.class);
   }
 
+  /** Converts the Java type used for UDF parameters of SQL DATE type
+   * ({@link java.sql.Date}) to internal representation (int).
+   *
+   * <p>Converse of {@link #internalToDate(int)}. */
   public static int toInt(java.util.Date v) {
     return toInt(v, LOCAL_TZ);
   }
@@ -1062,6 +1066,10 @@ public class SqlFunctions {
     return toLong(v, LOCAL_TZ);
   }
 
+  /** Converts the Java type used for UDF parameters of SQL TIME type
+   * ({@link java.sql.Time}) to internal representation (int).
+   *
+   * <p>Converse of {@link #internalToTime(int)}. */
   public static int toInt(java.sql.Time v) {
     return (int) (toLong(v) % DateTimeUtils.MILLIS_PER_DAY);
   }
@@ -1085,6 +1093,10 @@ public class SqlFunctions {
         : (Integer) cannotConvert(o, int.class);
   }
 
+  /** Converts the Java type used for UDF parameters of SQL TIMESTAMP type
+   * ({@link java.sql.Timestamp}) to internal representation (long).
+   *
+   * <p>Converse of {@link #internalToTimestamp(long)}. */
   public static long toLong(Timestamp v) {
     return toLong(v, LOCAL_TZ);
   }
@@ -1175,33 +1187,34 @@ public class SqlFunctions {
 
   /** Converts the internal representation of a SQL DATE (int) to the Java
    * type used for UDF parameters ({@link java.sql.Date}). */
-  public static java.sql.Date internalToDate(int x) {
-    return new java.sql.Date(x * DateTimeUtils.MILLIS_PER_DAY);
+  public static java.sql.Date internalToDate(int v) {
+    final long t = v * DateTimeUtils.MILLIS_PER_DAY;
+    return new java.sql.Date(t - LOCAL_TZ.getOffset(t));
   }
 
   /** As {@link #internalToDate(int)} but allows nulls. */
-  public static java.sql.Date internalToDate(Integer x) {
-    return x == null ? null : internalToDate(x.intValue());
+  public static java.sql.Date internalToDate(Integer v) {
+    return v == null ? null : internalToDate(v.intValue());
   }
 
   /** Converts the internal representation of a SQL TIME (int) to the Java
    * type used for UDF parameters ({@link java.sql.Time}). */
-  public static java.sql.Time internalToTime(int x) {
-    return new java.sql.Time(x);
+  public static java.sql.Time internalToTime(int v) {
+    return new java.sql.Time(v - LOCAL_TZ.getOffset(v));
   }
 
-  public static java.sql.Time internalToTime(Integer x) {
-    return x == null ? null : internalToTime(x.intValue());
+  public static java.sql.Time internalToTime(Integer v) {
+    return v == null ? null : internalToTime(v.intValue());
   }
 
   /** Converts the internal representation of a SQL TIMESTAMP (long) to the Java
    * type used for UDF parameters ({@link java.sql.Timestamp}). */
-  public static java.sql.Timestamp internalToTimestamp(long x) {
-    return new java.sql.Timestamp(x);
+  public static java.sql.Timestamp internalToTimestamp(long v) {
+    return new java.sql.Timestamp(v - LOCAL_TZ.getOffset(v));
   }
 
-  public static java.sql.Timestamp internalToTimestamp(Long x) {
-    return x == null ? null : internalToTimestamp(x.longValue());
+  public static java.sql.Timestamp internalToTimestamp(Long v) {
+    return v == null ? null : internalToTimestamp(v.longValue());
   }
 
   // Don't need shortValueOf etc. - Short.valueOf is sufficient.

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/runtime/Utilities.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Utilities.java b/core/src/main/java/org/apache/calcite/runtime/Utilities.java
index 691afd5..cf4fc0c 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Utilities.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Utilities.java
@@ -29,6 +29,8 @@ public class Utilities {
   protected Utilities() {
   }
 
+  /** @deprecated Use {@link java.util.Objects#equals}. */
+  @Deprecated // to be removed before 2.0
   public static boolean equal(Object o0, Object o1) {
     // Same as java.lang.Objects.equals (JDK 1.7 and later)
     // and com.google.common.base.Objects.equal

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/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 66fb911..87e73e0 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -493,7 +493,7 @@ public abstract class SqlUtil {
         final RelDataType argType = p.right;
         final RelDataType paramType = p.left;
         if (argType != null
-            && !SqlTypeUtil.canAssignFrom(paramType, argType)) {
+            && !SqlTypeUtil.canCastFrom(paramType, argType, false)) {
           iter.remove();
           continue loop;
         }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
index 1da123d..4fe315b 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
@@ -71,6 +71,7 @@ public class SqlTypeExplicitPrecedenceList
       ImmutableMap.<SqlTypeName, SqlTypeExplicitPrecedenceList>builder()
           .put(SqlTypeName.BOOLEAN, list(SqlTypeName.BOOLEAN))
           .put(SqlTypeName.TINYINT, numeric(SqlTypeName.TINYINT))
+          .put(SqlTypeName.SMALLINT, numeric(SqlTypeName.SMALLINT))
           .put(SqlTypeName.INTEGER, numeric(SqlTypeName.INTEGER))
           .put(SqlTypeName.BIGINT, numeric(SqlTypeName.BIGINT))
           .put(SqlTypeName.DECIMAL, numeric(SqlTypeName.DECIMAL))

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
index 658acd4..204b7ea 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
@@ -589,6 +589,7 @@ public abstract class SqlTypeUtil {
    * @return true if type has a representation as a Java primitive (ignoring
    * nullability)
    */
+  @Deprecated // to be removed before 2.0
   public static boolean isJavaPrimitive(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
@@ -614,6 +615,7 @@ public abstract class SqlTypeUtil {
   /**
    * @return class name of the wrapper for the primitive data type.
    */
+  @Deprecated // to be removed before 2.0
   public static String getPrimitiveWrapperJavaClassName(RelDataType type) {
     if (type == null) {
       return null;
@@ -627,6 +629,7 @@ public abstract class SqlTypeUtil {
     case BOOLEAN:
       return "Boolean";
     default:
+      //noinspection deprecation
       return getNumericJavaClassName(type);
     }
   }
@@ -634,6 +637,7 @@ public abstract class SqlTypeUtil {
   /**
    * @return class name of the numeric data type.
    */
+  @Deprecated // to be removed before 2.0
   public static String getNumericJavaClassName(RelDataType type) {
     if (type == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/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 4c7b41c..c899cb8 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -56,6 +56,7 @@ import org.apache.calcite.runtime.FlatLists;
 import org.apache.calcite.runtime.ResultSetEnumerable;
 import org.apache.calcite.runtime.SortedMultiMap;
 import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.runtime.Utilities;
 import org.apache.calcite.schema.FilterableTable;
 import org.apache.calcite.schema.ModifiableTable;
 import org.apache.calcite.schema.ProjectableFilterableTable;
@@ -82,6 +83,7 @@ import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.TimeZone;
 import javax.sql.DataSource;
 
@@ -254,10 +256,15 @@ public enum BuiltInMethod {
   IS_TRUE(SqlFunctions.class, "isTrue", Boolean.class),
   IS_NOT_FALSE(SqlFunctions.class, "isNotFalse", Boolean.class),
   NOT(SqlFunctions.class, "not", Boolean.class),
+  LESSER(SqlFunctions.class, "lesser", Comparable.class, Comparable.class),
+  GREATER(SqlFunctions.class, "greater", Comparable.class, Comparable.class),
   MODIFIABLE_TABLE_GET_MODIFIABLE_COLLECTION(ModifiableTable.class,
       "getModifiableCollection"),
   SCANNABLE_TABLE_SCAN(ScannableTable.class, "scan", DataContext.class),
   STRING_TO_BOOLEAN(SqlFunctions.class, "toBoolean", String.class),
+  INTERNAL_TO_DATE(SqlFunctions.class, "internalToDate", int.class),
+  INTERNAL_TO_TIME(SqlFunctions.class, "internalToTime", int.class),
+  INTERNAL_TO_TIMESTAMP(SqlFunctions.class, "internalToTimestamp", long.class),
   STRING_TO_DATE(DateTimeUtils.class, "dateStringToUnixDate", String.class),
   STRING_TO_TIME(DateTimeUtils.class, "timeStringToUnixDate", String.class),
   STRING_TO_TIMESTAMP(DateTimeUtils.class, "timestampStringToUnixDate",
@@ -288,8 +295,13 @@ public enum BuiltInMethod {
   BOOLEAN_TO_STRING(SqlFunctions.class, "toString", boolean.class),
   JDBC_ARRAY_TO_LIST(SqlFunctions.class, "arrayToList", java.sql.Array.class),
   OBJECT_TO_STRING(Object.class, "toString"),
-  OBJECTS_EQUAL(com.google.common.base.Objects.class, "equal", Object.class,
-      Object.class),
+  OBJECTS_EQUAL(Objects.class, "equals", Object.class, Object.class),
+  HASH(Utilities.class, "hash", int.class, Object.class),
+  COMPARE(Utilities.class, "compare", Comparable.class, Comparable.class),
+  COMPARE_NULLS_FIRST(Utilities.class, "compareNullsFirst", Comparable.class,
+      Comparable.class),
+  COMPARE_NULLS_LAST(Utilities.class, "compareNullsLast", Comparable.class,
+      Comparable.class),
   ROUND_LONG(SqlFunctions.class, "round", long.class, long.class),
   ROUND_INT(SqlFunctions.class, "round", int.class, int.class),
   DATE_TO_INT(SqlFunctions.class, "toInt", java.util.Date.class),

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
index f9e2bea..4b36b51 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -275,7 +275,8 @@ public class CalciteAssert {
             throw new AssertionError("expected 1 column");
           }
           final String resultString = resultSet.getString(1);
-          assertEquals(expected, Util.toLinux(resultString));
+          assertEquals(expected,
+              resultString == null ? null : Util.toLinux(resultString));
           return null;
         } catch (SQLException e) {
           throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/test/java/org/apache/calcite/test/UdfTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/UdfTest.java b/core/src/test/java/org/apache/calcite/test/UdfTest.java
index 7d707fe..a8d7766 100644
--- a/core/src/test/java/org/apache/calcite/test/UdfTest.java
+++ b/core/src/test/java/org/apache/calcite/test/UdfTest.java
@@ -105,6 +105,12 @@ public class UdfTest {
         + "         {\n"
         + "           name: 'COUNT_ARGS',\n"
         + "           className: '"
+        + Smalls.CountArgs1NullableFunction.class.getName()
+        + "'\n"
+        + "         },\n"
+        + "         {\n"
+        + "           name: 'COUNT_ARGS',\n"
+        + "           className: '"
         + Smalls.CountArgs2Function.class.getName()
         + "'\n"
         + "         },\n"
@@ -261,6 +267,14 @@ public class UdfTest {
         .returns("P0=0; P1=1; P2=2\n");
   }
 
+  @Test public void testUdfOverloadedNullable() {
+    final CalciteAssert.AssertThat with = withUdf();
+    with.query("values (\"adhoc\".count_args(),\n"
+        + " \"adhoc\".count_args(cast(null as smallint)),\n"
+        + " \"adhoc\".count_args(0, 0))")
+        .returns("EXPR$0=0; EXPR$1=-1; EXPR$2=2\n");
+  }
+
   /** Tests passing parameters to user-defined function by name. */
   @Test public void testUdfArgumentName() {
     final CalciteAssert.AssertThat with = withUdf();
@@ -586,6 +600,54 @@ public class UdfTest {
     with.query("values \"adhoc\".\"timestampFun\"(cast(null as timestamp))")
         .returnsValue("-1");
   }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1041">[CALCITE-1041]
+   * User-defined function returns DATE or TIMESTAMP value</a>. */
+  @Test public void testReturnDate() {
+    final CalciteAssert.AssertThat with = withUdf();
+    with.query("values \"adhoc\".\"toDateFun\"(0)")
+        .returnsValue("1970-01-01");
+    with.query("values \"adhoc\".\"toDateFun\"(1)")
+        .returnsValue("1970-01-02");
+    with.query("values \"adhoc\".\"toDateFun\"(cast(null as bigint))")
+        .returnsValue(null);
+    with.query("values \"adhoc\".\"toTimeFun\"(0)")
+        .returnsValue("00:00:00");
+    with.query("values \"adhoc\".\"toTimeFun\"(90000)")
+        .returnsValue("00:01:30");
+    with.query("values \"adhoc\".\"toTimeFun\"(cast(null as bigint))")
+        .returnsValue(null);
+    with.query("values \"adhoc\".\"toTimestampFun\"(0)")
+        .returnsValue("1970-01-01 00:00:00");
+    with.query("values \"adhoc\".\"toTimestampFun\"(86490000)")
+        .returnsValue("1970-01-02 00:01:30");
+    with.query("values \"adhoc\".\"toTimestampFun\"(cast(null as bigint))")
+        .returnsValue(null);
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1041">[CALCITE-1041]
+   * User-defined function returns DATE or TIMESTAMP value</a>. */
+  @Test public void testReturnDate2() {
+    final CalciteAssert.AssertThat with = withUdf();
+    with.query("select * from (values 0) as t(c)\n"
+        + "where \"adhoc\".\"toTimestampFun\"(c) in (\n"
+        + "  cast('1970-01-01 00:00:00' as timestamp),\n"
+        + "  cast('1997-02-01 00:00:00' as timestamp))")
+        .returnsValue("0");
+    with.query("select * from (values 0) as t(c)\n"
+        + "where \"adhoc\".\"toTimestampFun\"(c) in (\n"
+        + "  timestamp '1970-01-01 00:00:00',\n"
+        + "  timestamp '1997-02-01 00:00:00')")
+        .returnsValue("0");
+    with.query("select * from (values 0) as t(c)\n"
+        + "where \"adhoc\".\"toTimestampFun\"(c) in (\n"
+        + "  '1970-01-01 00:00:00',\n"
+        + "  '1997-02-01 00:00:00')")
+        .returnsValue("0");
+  }
+
 }
 
 // End UdfTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/test/java/org/apache/calcite/util/Smalls.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/Smalls.java b/core/src/test/java/org/apache/calcite/util/Smalls.java
index 62cc064..d9e07ef 100644
--- a/core/src/test/java/org/apache/calcite/util/Smalls.java
+++ b/core/src/test/java/org/apache/calcite/util/Smalls.java
@@ -33,6 +33,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.schema.QueryableTable;
 import org.apache.calcite.schema.ScannableTable;
 import org.apache.calcite.schema.SchemaPlus;
@@ -418,20 +419,34 @@ public class Smalls {
   public static class AllTypesFunction {
     private AllTypesFunction() {}
 
-    public static long dateFun(java.sql.Date x) { return x == null ? -1L : x.getTime(); }
-    public static long timestampFun(java.sql.Timestamp x) { return x == null ? -1L : x.getTime(); }
-    public static long timeFun(java.sql.Time x) { return x == null ? -1L : x.getTime(); }
+    // We use SqlFunctions.toLong(Date) ratter than Date.getTime(),
+    // and SqlFunctions.internalToTimestamp(long) rather than new Date(long),
+    // because the contract of JDBC (also used by UDFs) is to represent
+    // date-time values in the LOCAL time zone.
 
-    public static java.sql.Date toDateFun(int x) { return new java.sql.Date(x); }
+    public static long dateFun(java.sql.Date v) {
+      return v == null ? -1L : SqlFunctions.toLong(v);
+    }
+    public static long timestampFun(java.sql.Timestamp v) {
+      return v == null ? -1L : SqlFunctions.toLong(v);
+    }
+    public static long timeFun(java.sql.Time v) {
+      return v == null ? -1L : SqlFunctions.toLong(v);
+    }
+
+    /** Overloaded, in a challenging way, with {@link #toDateFun(Long)}. */
+    public static java.sql.Date toDateFun(int v) {
+      return SqlFunctions.internalToDate(v);
+    }
 
-    public static java.sql.Date toDateFun(Long x) {
-      return x == null ? null : new java.sql.Date(x);
+    public static java.sql.Date toDateFun(Long v) {
+      return v == null ? null : SqlFunctions.internalToDate(v.intValue());
     }
-    public static java.sql.Timestamp toTimestampFun(Long x) {
-      return x == null ? null : new java.sql.Timestamp(x);
+    public static java.sql.Timestamp toTimestampFun(Long v) {
+      return SqlFunctions.internalToTimestamp(v);
     }
-    public static java.sql.Time toTimeFun(Long x) {
-      return x == null ? null : new java.sql.Time(x);
+    public static java.sql.Time toTimeFun(Long v) {
+      return v == null ? null : SqlFunctions.internalToTime(v.intValue());
     }
   }