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 2015/12/05 04:49:51 UTC

[2/2] calcite git commit: Not safe to use '=' for predicates on constant expressions that might be null

Not safe to use '=' for predicates on constant expressions that might be null

Constants of all types, not just literals and calls, can be
converted to predicates (taking care if they might be null).

Make RexProgram.ConstantFinder extend RexUtil.ConstantFinder.

Make test deterministic even if predicates occur in different order.


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

Branch: refs/heads/master
Commit: 8281668fbf7b5f1cc5f64796ebe1604ae634510d
Parents: 9be2bdb
Author: Julian Hyde <jh...@apache.org>
Authored: Fri Dec 4 13:29:10 2015 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Dec 4 18:02:51 2015 -0800

----------------------------------------------------------------------
 .../calcite/rel/metadata/RelMdPredicates.java   | 23 +++++----
 .../java/org/apache/calcite/rex/RexProgram.java | 43 ++--------------
 .../java/org/apache/calcite/rex/RexUtil.java    | 10 ++--
 .../apache/calcite/test/RelMetadataTest.java    | 52 +++++++++++++++++---
 4 files changed, 67 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/8281668f/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
index a35d571..7b20b68 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
@@ -41,6 +41,7 @@ import org.apache.calcite.rex.RexPermuteInputsShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.BuiltInMethod;
@@ -185,16 +186,14 @@ public class RelMdPredicates {
         projectPullUpPredicates.add(
             rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL,
                 rexBuilder.makeInputRef(project, expr.i)));
-      } else if (expr.e instanceof RexLiteral) {
-        final RexLiteral literal = (RexLiteral) expr.e;
-        projectPullUpPredicates.add(
-            rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
-                rexBuilder.makeInputRef(project, expr.i), literal));
-      } else if (expr.e instanceof RexCall
-                && RexUtil.isConstant(expr.e)) {
-        projectPullUpPredicates.add(
-            rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
-            rexBuilder.makeInputRef(project, expr.i), expr.e));
+      } else if (RexUtil.isConstant(expr.e)) {
+        final List<RexNode> args =
+            ImmutableList.of(rexBuilder.makeInputRef(project, expr.i), expr.e);
+        final SqlOperator op = args.get(0).getType().isNullable()
+            || args.get(1).getType().isNullable()
+            ? SqlStdOperatorTable.IS_NOT_DISTINCT_FROM
+            : SqlStdOperatorTable.EQUALS;
+        projectPullUpPredicates.add(rexBuilder.makeCall(op, args));
       }
     }
     return RelOptPredicateList.of(projectPullUpPredicates);
@@ -371,8 +370,8 @@ public class RelMdPredicates {
     final RexNode leftChildPredicates;
     final RexNode rightChildPredicates;
 
-    JoinConditionBasedPredicateInference(Join joinRel, RexNode lPreds,
-                                         RexNode rPreds) {
+    public JoinConditionBasedPredicateInference(Join joinRel,
+            RexNode lPreds, RexNode rPreds) {
       this(joinRel, joinRel instanceof SemiJoin, lPreds, rPreds);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/8281668f/core/src/main/java/org/apache/calcite/rex/RexProgram.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexProgram.java b/core/src/main/java/org/apache/calcite/rex/RexProgram.java
index caef1de..6fd12ce 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgram.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgram.java
@@ -853,54 +853,17 @@ public class RexProgram {
   /**
    * Walks over an expression and determines whether it is constant.
    */
-  private class ConstantFinder implements RexVisitor<Boolean> {
-    private ConstantFinder() {
-    }
-
-    public Boolean visitLiteral(RexLiteral literal) {
-      return true;
-    }
-
-    public Boolean visitInputRef(RexInputRef inputRef) {
-      return false;
-    }
-
-    public Boolean visitLocalRef(RexLocalRef localRef) {
+  private class ConstantFinder extends RexUtil.ConstantFinder {
+    @Override public Boolean visitLocalRef(RexLocalRef localRef) {
       final RexNode expr = exprs.get(localRef.index);
       return expr.accept(this);
     }
 
-    public Boolean visitOver(RexOver over) {
-      return false;
-    }
-
-    public Boolean visitCorrelVariable(RexCorrelVariable correlVariable) {
+    @Override public Boolean visitCorrelVariable(RexCorrelVariable correlVariable) {
       // Correlating variables are constant WITHIN A RESTART, so that's
       // good enough.
       return true;
     }
-
-    public Boolean visitDynamicParam(RexDynamicParam dynamicParam) {
-      // Dynamic parameters are constant WITHIN A RESTART, so that's
-      // good enough.
-      return true;
-    }
-
-    public Boolean visitCall(RexCall call) {
-      // Constant if operator is deterministic and all operands are
-      // constant.
-      return call.getOperator().isDeterministic()
-          && RexVisitorImpl.visitArrayAnd(this, call.getOperands());
-    }
-
-    public Boolean visitRangeRef(RexRangeRef rangeRef) {
-      return false;
-    }
-
-    public Boolean visitFieldAccess(RexFieldAccess fieldAccess) {
-      // "<expr>.FIELD" is constant iff "<expr>" is constant.
-      return fieldAccess.getReferenceExpr().accept(this);
-    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/8281668f/core/src/main/java/org/apache/calcite/rex/RexUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index 6aaf377..7307654 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -240,7 +240,7 @@ public class RexUtil {
   /**
    * Walks over an expression and determines whether it is constant.
    */
-  private static class ConstantFinder implements RexVisitor<Boolean> {
+  static class ConstantFinder implements RexVisitor<Boolean> {
     static final ConstantFinder INSTANCE = new ConstantFinder();
 
     public Boolean visitLiteral(RexLiteral literal) {
@@ -260,10 +260,14 @@ public class RexUtil {
     }
 
     public Boolean visitCorrelVariable(RexCorrelVariable correlVariable) {
+      // Correlating variables change when there is an internal restart.
+      // Not good enough for our purposes.
       return false;
     }
 
     public Boolean visitDynamicParam(RexDynamicParam dynamicParam) {
+      // Dynamic parameters are constant WITHIN AN EXECUTION, so that's
+      // good enough.
       return true;
     }
 
@@ -287,7 +291,7 @@ public class RexUtil {
   /**
    * Returns whether node is made up of constants.
    *
-   * @param node to inspect
+   * @param node Node to inspect
    * @return true if node is made up of constants, false otherwise
    */
   public static boolean isConstant(RexNode node) {
@@ -297,7 +301,7 @@ public class RexUtil {
    /**
    * Returns whether a given node contains a RexCall with a specified operator
    *
-   * @param operator to look for
+   * @param operator Operator to look for
    * @param node     a RexNode tree
    */
   public static RexCall findOperatorCall(

http://git-wip-us.apache.org/repos/asf/calcite/blob/8281668f/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
index ddae101..0e76a40 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -72,13 +72,16 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 
 import org.hamcrest.CoreMatchers;
+import org.hamcrest.CustomTypeSafeMatcher;
 import org.hamcrest.Matcher;
 import org.junit.Ignore;
 import org.junit.Test;
 
 import java.lang.reflect.Method;
 import java.math.BigDecimal;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
@@ -1250,8 +1253,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
                     + empScan.getRowType().getFieldCount()));
 
     predicates = RelMetadataQuery.getPulledUpPredicates(semiJoin);
-    assertThat(predicates.pulledUpPredicates.toString(), is("[=($0, 1)]"));
-    assertThat(predicates.leftInferredPredicates.toString(), is("[]"));
+    assertThat(predicates.pulledUpPredicates, sortsAs("[=($0, 1)]"));
+    assertThat(predicates.leftInferredPredicates, sortsAs("[]"));
     assertThat(predicates.rightInferredPredicates.isEmpty(), is(true));
   }
 
@@ -1266,18 +1269,55 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final Aggregate rel = (Aggregate) convertSql(sql);
     RelOptPredicateList inputSet = RelMetadataQuery.getPulledUpPredicates(rel);
     ImmutableList<RexNode> pulledUpPredicates = inputSet.pulledUpPredicates;
-    assertThat(pulledUpPredicates.toString(), is("[=($0, 1)]"));
+    assertThat(pulledUpPredicates, sortsAs("[=($0, 1)]"));
   }
 
-  @Test public void testPullUpPredicatesFromProject() {
+  @Test public void testPullUpPredicatesOnConstant() {
     final String sql = "select deptno, mgr, x, 'y' as y, z from (\n"
         + "  select deptno, mgr, cast(null as integer) as x, cast('1' as int) as z\n"
         + "  from emp\n"
         + "  where mgr is null and deptno < 10)";
     final RelNode rel = convertSql(sql);
     RelOptPredicateList list = RelMetadataQuery.getPulledUpPredicates(rel);
-    assertThat(list.pulledUpPredicates.toString(),
-        is("[IS NULL($1), <($0, 10), IS NULL($2), =($4, CAST('1'):INTEGER NOT NULL), =($3, 'y')]"));
+    assertThat(list.pulledUpPredicates,
+        sortsAs("[<($0, 10), =($3, 'y'), =($4, CAST('1'):INTEGER NOT NULL), "
+            + "IS NULL($1), IS NULL($2)]"));
+  }
+
+  @Test public void testPullUpPredicatesOnNullableConstant() {
+    final String sql = "select nullif(1, 1) as c\n"
+        + "  from emp\n"
+        + "  where mgr is null and deptno < 10";
+    final RelNode rel = convertSql(sql);
+    RelOptPredicateList list = RelMetadataQuery.getPulledUpPredicates(rel);
+    // Uses "IS NOT DISTINCT FROM" rather than "=" because cannot guarantee not null.
+    assertThat(list.pulledUpPredicates,
+        sortsAs("[IS NOT DISTINCT FROM($0, CASE(=(1, 1), null, 1))]"));
+  }
+
+  /**
+   * Matcher that succeeds for any collection that, when converted to strings
+   * and sorted on those strings, matches the given reference string.
+   *
+   * <p>Use it as an alternative to {@link CoreMatchers#is} if items in your
+   * list might occur in any order.
+   *
+   * <p>For example:
+   *
+   * <pre>List&lt;Integer&gt; ints = Arrays.asList(2, 500, 12);
+   * assertThat(ints, sortsAs("[12, 2, 500]");</pre>
+   */
+  static <T> Matcher<Iterable<? extends T>> sortsAs(final String value) {
+    return new CustomTypeSafeMatcher<Iterable<? extends T>>(value) {
+      protected boolean matchesSafely(Iterable<? extends T> item) {
+        final List<String> strings = new ArrayList<>();
+        for (T t : item) {
+          strings.add(t.toString());
+        }
+        Collections.sort(strings);
+        return value.equals(strings.toString());
+      }
+    };
   }
 
   /** Custom metadata interface. */