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/21 23:38:41 UTC

[04/50] [abbrv] calcite git commit: [CALCITE-1042] Ensure that FILTER is BOOLEAN NOT NULL

[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/branch-release
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);
   }
 
   /**