You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by as...@apache.org on 2023/04/25 14:29:08 UTC

[calcite] branch main updated: [CALCITE-5639] RexSimplify should remove IS NOT NULL check when LIKE comparison is present

This is an automated email from the ASF dual-hosted git repository.

asolimando pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/calcite.git


The following commit(s) were added to refs/heads/main by this push:
     new 0305067459 [CALCITE-5639] RexSimplify should remove IS NOT NULL check when LIKE comparison is present
0305067459 is described below

commit 03050674594152ea785af69517f33960d7e27dd5
Author: Leonid Chistov <lc...@querifylabs.com>
AuthorDate: Tue Apr 11 16:02:59 2023 +0300

    [CALCITE-5639] RexSimplify should remove IS NOT NULL check when LIKE comparison is present
---
 .../main/java/org/apache/calcite/plan/Strong.java  | 19 +++++
 .../java/org/apache/calcite/rex/RexSimplify.java   | 97 +++++++++++++---------
 .../apache/calcite/rex/RexProgramBuilderBase.java  |  4 +
 .../org/apache/calcite/rex/RexProgramTest.java     | 51 ++++++++++++
 core/src/test/resources/sql/sub-query.iq           |  2 +-
 5 files changed, 134 insertions(+), 39 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/plan/Strong.java b/core/src/main/java/org/apache/calcite/plan/Strong.java
index 935e7c8f6a..e2b67f0762 100644
--- a/core/src/main/java/org/apache/calcite/plan/Strong.java
+++ b/core/src/main/java/org/apache/calcite/plan/Strong.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.plan;
 
 import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexFieldAccess;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
@@ -27,6 +28,7 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.util.ImmutableBitSet;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 
 import java.util.ArrayList;
@@ -72,6 +74,16 @@ public class Strong {
     };
   }
 
+  /** Returns a checker that consults a set to find out whether particular
+   * field may be null. */
+  public static Strong of(final ImmutableSet<RexFieldAccess> nullFields) {
+    return new Strong() {
+      @Override public boolean isNull(RexFieldAccess ref) {
+        return nullFields.contains(ref);
+      }
+    };
+  }
+
   /** Returns whether the analyzed expression will definitely return null if
    * all of a given set of input columns are null. */
   public static boolean isNull(RexNode node, ImmutableBitSet nullColumns) {
@@ -216,6 +228,8 @@ public class Strong {
       return allNull(ImmutableList.of(((RexCall) node).getOperands().get(0)));
     case INPUT_REF:
       return isNull((RexInputRef) node);
+    case FIELD_ACCESS:
+      return isNull((RexFieldAccess) node);
     case CASE:
       final RexCall caseCall = (RexCall) node;
       final List<RexNode> caseValues = new ArrayList<>();
@@ -235,6 +249,11 @@ public class Strong {
     return false;
   }
 
+  /** Returns whether a given field is definitely null. */
+  public boolean isNull(RexFieldAccess ref) {
+    return false;
+  }
+
   /** Returns whether all expressions in a list are definitely null. */
   private boolean allNull(List<RexNode> operands) {
     for (RexNode operand : operands) {
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
index 38b2b2b040..c95752b80c 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -32,6 +32,7 @@ import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.RangeSets;
 import org.apache.calcite.util.Sarg;
@@ -41,6 +42,7 @@ import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.BoundType;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableRangeSet;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Range;
@@ -1599,7 +1601,6 @@ public class RexSimplify {
     final Set<RexNode> negatedTerms = new HashSet<>();
     final Set<RexNode> nullOperands = new HashSet<>();
     final Set<RexNode> notNullOperands = new LinkedHashSet<>();
-    final Set<RexNode> comparedOperands = new HashSet<>();
 
     // Add the predicates from the source to the range terms.
     for (RexNode predicate : predicates.pulledUpPredicates) {
@@ -1647,19 +1648,7 @@ public class RexSimplify {
       case GREATER_THAN_OR_EQUAL:
         RexCall call = (RexCall) term;
         final RexNode left = call.getOperands().get(0);
-        comparedOperands.add(left);
-        // if it is a cast, we include the inner reference
-        if (left.getKind() == SqlKind.CAST) {
-          RexCall leftCast = (RexCall) left;
-          comparedOperands.add(leftCast.getOperands().get(0));
-        }
         final RexNode right = call.getOperands().get(1);
-        comparedOperands.add(right);
-        // if it is a cast, we include the inner reference
-        if (right.getKind() == SqlKind.CAST) {
-          RexCall rightCast = (RexCall) right;
-          comparedOperands.add(rightCast.getOperands().get(0));
-        }
         final Comparison comparison = Comparison.of(term);
         // Check for comparison with null values
         if (comparison != null
@@ -1719,12 +1708,6 @@ public class RexSimplify {
           }
         }
         break;
-      case IN:
-        comparedOperands.add(((RexCall) term).operands.get(0));
-        break;
-      case BETWEEN:
-        comparedOperands.add(((RexCall) term).operands.get(1));
-        break;
       case IS_NOT_NULL:
         notNullOperands.add(((RexCall) term).getOperands().get(0));
         terms.remove(i);
@@ -1737,12 +1720,6 @@ public class RexSimplify {
         break;
       }
     }
-    // If one column should be null and is in a comparison predicate,
-    // it is not satisfiable.
-    // Example. IS NULL(x) AND x < 5  - not satisfiable
-    if (!Collections.disjoint(nullOperands, comparedOperands)) {
-      return rexBuilder.makeLiteral(false);
-    }
     // Check for equality of two refs wrt equality with constants
     // Example #1. x=5 AND y=5 AND x=y : x=5 AND y=5
     // Example #2. x=5 AND y=6 AND x=y - not satisfiable
@@ -1767,15 +1744,6 @@ public class RexSimplify {
         terms.remove(ref2.right);
       }
     }
-    // Remove not necessary IS NOT NULL expressions.
-    //
-    // Example. IS NOT NULL(x) AND x < 5  : x < 5
-    for (RexNode operand : notNullOperands) {
-      if (!comparedOperands.contains(operand)) {
-        terms.add(
-            rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, operand));
-      }
-    }
     // If one of the not-disjunctions is a disjunction that is wholly
     // contained in the disjunctions list, the expression is not
     // satisfiable.
@@ -1793,16 +1761,53 @@ public class RexSimplify {
         return rexBuilder.makeLiteral(false);
       }
     }
-    // Add the NOT disjunctions back in.
-    for (RexNode notDisjunction : notTerms) {
-      terms.add(not(notDisjunction));
-    }
     // The negated terms: only deterministic expressions
     for (RexNode negatedTerm : negatedTerms) {
       if (termsSet.contains(negatedTerm)) {
         return rexBuilder.makeLiteral(false);
       }
     }
+    // Add the NOT disjunctions back in.
+    for (RexNode notDisjunction : notTerms) {
+      terms.add(not(notDisjunction));
+    }
+    // Find operands that make will let whole expression evaluate to FALSE if set to NULL
+    final Set<RexNode> strongOperands = new HashSet<>();
+    for (RexNode term : terms) {
+      if (!RexUtil.isDeterministic(term)) {
+        continue;
+      }
+      final VariableCollector collector = new VariableCollector();
+      term.accept(collector);
+      for (RexInputRef ref : collector.refs) {
+        final boolean strong = Strong.isNotTrue(term, ImmutableBitSet.of(ref.index));
+        if (strong) {
+          strongOperands.add(ref);
+        }
+      }
+      final RexUtil.FieldAccessFinder fieldAccessFinder = new RexUtil.FieldAccessFinder();
+      term.accept(fieldAccessFinder);
+      for (RexFieldAccess rexFieldAccess : fieldAccessFinder.getFieldAccessList()) {
+        final boolean strong = Strong.of(ImmutableSet.of(rexFieldAccess)).isNotTrue(term);
+        if (strong) {
+          strongOperands.add(rexFieldAccess);
+        }
+      }
+    }
+    // If one column should be null and is in a comparison predicate,
+    // it is not satisfiable.
+    // Example. IS NULL(x) AND x < 5  - not satisfiable
+    if (!Collections.disjoint(nullOperands, strongOperands)) {
+      return rexBuilder.makeLiteral(false);
+    }
+    // Remove not necessary IS NOT NULL expressions.
+    // Example. IS NOT NULL(x) AND x < 5  : x < 5
+    for (RexNode operand : notNullOperands) {
+      if (!strongOperands.contains(operand)) {
+        terms.add(
+            rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, operand));
+      }
+    }
     return RexUtil.composeConjunction(rexBuilder, terms);
   }
 
@@ -2599,6 +2604,22 @@ public class RexSimplify {
     }
   }
 
+  /**
+   * Visitor which finds all inputs used by an expressions.
+   */
+  private static class VariableCollector extends RexVisitorImpl<Void> {
+    private final Set<RexInputRef> refs = new HashSet<>();
+
+    VariableCollector() {
+      super(true);
+    }
+
+    @Override public Void visitInputRef(RexInputRef inputRef) {
+      refs.add(inputRef);
+      return super.visitInputRef(inputRef);
+    }
+  }
+
   /** Represents a simple Comparison.
    *
    * <p>Left hand side is a {@link RexNode}, right hand side is a literal.
diff --git a/core/src/test/java/org/apache/calcite/rex/RexProgramBuilderBase.java b/core/src/test/java/org/apache/calcite/rex/RexProgramBuilderBase.java
index dfa1269add..5273a86ebb 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramBuilderBase.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramBuilderBase.java
@@ -288,6 +288,10 @@ public abstract class RexProgramBuilderBase {
     return rexBuilder.makeCall(SqlStdOperatorTable.LIKE, ref, pattern);
   }
 
+  protected RexNode similar(RexNode ref, RexNode pattern) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.SIMILAR_TO, ref, pattern);
+  }
+
   protected RexNode like(RexNode ref, RexNode pattern, RexNode escape) {
     return rexBuilder.makeCall(SqlStdOperatorTable.LIKE, ref, pattern, escape);
   }
diff --git a/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java b/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
index 6b094e1f54..3d86b759e9 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
@@ -3288,6 +3288,57 @@ class RexProgramTest extends RexProgramTestBase {
         not(rexBuilder.makeCall(SqlStdOperatorTable.SIMILAR_TO, ref, literal("%"))));
   }
 
+  @Test void testSimplifyNullCheckInFilter() {
+    final RexNode iRef = input(tInt(true), 0);
+    final RexNode vRef = input(tVarchar(true, 10), 1);
+    final RexNode iRef2 = input(tInt(true), 2);
+    final RexNode vRef2 = input(tVarchar(true, 10), 3);
+    final RexNode fieldRef = vVarchar();
+
+    // CAST(x) > 1 AND x IS NOT NULL
+    RelDataType longType =
+        typeFactory.createTypeWithNullability(
+            typeFactory.createSqlType(SqlTypeName.BIGINT), false);
+    checkSimplifyFilter(and(isNotNull(iRef), gt(cast(iRef, longType), literal(BigDecimal.ONE))),
+        ">(CAST($0):BIGINT NOT NULL, 1)");
+
+    // LIKE
+    checkSimplifyFilter(and(isNotNull(vRef), like(vRef, literal("%hello%"))),
+        "LIKE($1, '%hello%')");
+
+    // LIKE on field ref
+    checkSimplifyFilter(and(isNotNull(fieldRef), like(fieldRef, literal("%hello%"))),
+        "LIKE(?0.varchar0, '%hello%')");
+
+    // NOT LIKE
+    checkSimplifyFilter(and(isNotNull(vRef), not(like(vRef, literal("%hello%")))),
+        "NOT(LIKE($1, '%hello%'))");
+
+    // OR(LIKE, LIKE)
+    checkSimplifyFilter(
+        and(isNotNull(vRef),
+            or(like(vRef, literal("%hello%")), like(vRef, literal("%bye%")))),
+        "OR(LIKE($1, '%hello%'), LIKE($1, '%bye%'))");
+
+    // SIMILAR TO
+    checkSimplifyFilter(and(isNotNull(vRef), similar(vRef, literal("%hello%"))),
+        "SIMILAR TO($1, '%hello%')");
+
+    // Arithmetic expression compared to value and checked for not null
+    checkSimplifyFilter(and(isNotNull(mul(iRef, iRef2)), gt(mul(iRef, iRef2), literal(10))),
+        ">(*($0, $2), 10)");
+
+    // COALESCE: not simplified because expression is not strong
+    checkSimplifyFilter(and(isNotNull(vRef2), coalesce(vRef, vRef2)),
+        "AND(COALESCE($1, $3), IS NOT NULL($3))");
+
+    // Not simplified because expression is not strong
+    checkSimplifyFilter(
+        and(isNotNull(vRef),
+            or(like(vRef, literal("%hello%")), like(vRef2, literal("%bye%")))),
+        "AND(OR(LIKE($1, '%hello%'), LIKE($3, '%bye%')), IS NOT NULL($1))");
+  }
+
   @Test void testSimplifyNonDeterministicFunction() {
     final SqlOperator ndc =
         SqlBasicFunction.create("NDC", ReturnTypes.BOOLEAN,
diff --git a/core/src/test/resources/sql/sub-query.iq b/core/src/test/resources/sql/sub-query.iq
index 89b28976e2..277c8ef0b5 100644
--- a/core/src/test/resources/sql/sub-query.iq
+++ b/core/src/test/resources/sql/sub-query.iq
@@ -1748,7 +1748,7 @@ select sal from "scott".emp e
 (0 rows)
 
 !ok
-EnumerableCalc(expr#0..4=[{inputs}], expr#5=[RAND()], expr#6=[CAST($t5):INTEGER NOT NULL], expr#7=[2], expr#8=[MOD($t6, $t7)], expr#9=[3], expr#10=[=($t8, $t9)], expr#11=[IS NOT NULL($t4)], expr#12=[AND($t4, $t11)], expr#13=[OR($t10, $t12)], SAL=[$t1], $condition=[$t13])
+EnumerableCalc(expr#0..4=[{inputs}], expr#5=[RAND()], expr#6=[CAST($t5):INTEGER NOT NULL], expr#7=[2], expr#8=[MOD($t6, $t7)], expr#9=[3], expr#10=[=($t8, $t9)], expr#11=[OR($t10, $t4)], SAL=[$t1], $condition=[$t11])
   EnumerableMergeJoin(condition=[=($2, $3)], joinType=[left])
     EnumerableSort(sort0=[$2], dir0=[ASC])
       EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], SAL=[$t5], DEPTNO=[$t7])