You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by da...@apache.org on 2020/11/12 09:39:19 UTC

[calcite] branch master updated: [CALCITE-4364] `a IN (1, 2) AND a = 1` should be simplified to `a = 1`

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 5e9943a  [CALCITE-4364] `a IN (1, 2) AND a = 1` should be simplified to `a = 1`
5e9943a is described below

commit 5e9943aa1f51a97068fc37d53dea1d447570becc
Author: yuzhao.cyz <yu...@gmail.com>
AuthorDate: Thu Nov 5 15:01:59 2020 +0800

    [CALCITE-4364] `a IN (1, 2) AND a = 1` should be simplified to `a = 1`
---
 .../java/org/apache/calcite/rex/RexSimplify.java   |  90 ++++++++++++---
 .../main/java/org/apache/calcite/rex/RexUtil.java  |  30 ++++-
 .../java/org/apache/calcite/util/RangeSets.java    |  12 ++
 .../org/apache/calcite/rex/RexProgramTest.java     | 128 ++++++++++++++-------
 .../org/apache/calcite/test/RelBuilderTest.java    |   6 +-
 .../java/org/apache/calcite/util/RangeSetTest.java |  32 ++++++
 .../org/apache/calcite/test/RelOptRulesTest.xml    |  10 +-
 core/src/test/resources/sql/sub-query.iq           |   2 +-
 .../org/apache/calcite/test/DruidAdapter2IT.java   |   8 +-
 .../org/apache/calcite/test/DruidAdapterIT.java    |   8 +-
 10 files changed, 250 insertions(+), 76 deletions(-)

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 65b3ba3..06acfb5 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -1330,7 +1330,7 @@ public class RexSimplify {
 
     final SargCollector sargCollector = new SargCollector(rexBuilder, true);
     operands.forEach(t -> sargCollector.accept(t, terms));
-    if (sargCollector.map.values().stream().anyMatch(b -> b.complexity() > 1)) {
+    if (sargCollector.needToFix(unknownAs)) {
       operands.clear();
       terms.forEach(t -> operands.add(sargCollector.fix(rexBuilder, t)));
     }
@@ -1797,7 +1797,7 @@ public class RexSimplify {
     final SargCollector sargCollector = new SargCollector(rexBuilder, false);
     final List<RexNode> newTerms = new ArrayList<>();
     terms.forEach(t -> sargCollector.accept(t, newTerms));
-    if (sargCollector.map.values().stream().anyMatch(b -> b.complexity() > 1)) {
+    if (sargCollector.needToFix(unknownAs)) {
       terms.clear();
       newTerms.forEach(t -> terms.add(sargCollector.fix(rexBuilder, t)));
     }
@@ -2591,6 +2591,12 @@ public class RexSimplify {
     final Map<RexNode, RexSargBuilder> map = new HashMap<>();
     private final RexBuilder rexBuilder;
     private final boolean negate;
+    /**
+     * Count of the new terms after converting all the operands to
+     * {@code SEARCH} on a {@link Sarg}. It is used to decide whether
+     * the new terms are simpler.
+     */
+    private int newTermsCount;
 
     SargCollector(RexBuilder rexBuilder, boolean negate) {
       this.rexBuilder = rexBuilder;
@@ -2601,6 +2607,7 @@ public class RexSimplify {
       if (!accept_(term, newTerms)) {
         newTerms.add(term);
       }
+      newTermsCount = newTerms.size();
     }
 
     private boolean accept_(RexNode e, List<RexNode> newTerms) {
@@ -2710,14 +2717,76 @@ public class RexSimplify {
       }
     }
 
+    /**
+     * Returns whether the merged {@code sarg} with given {@code unknownAs} keeps the semantics,
+     * The merge can not go ahead if the semantics change.
+     *
+     * @return true if the semantics does not change
+     */
+    private static boolean canMerge(Sarg sarg, RexUnknownAs unknownAs) {
+      final boolean isAllOrNone = sarg.isAll() || sarg.isNone();
+      final boolean containsNull = sarg.containsNull;
+      switch (unknownAs) {
+      case UNKNOWN:
+        // "unknown as unknown" can not be simplified to
+        // "IS NULL"/"IS NOT NULL"/"TRUE"/"FALSE"
+        return !isAllOrNone;
+      case TRUE:
+        // "unknown as true" can not be simplified to
+        // "false" or "IS NOT NULL"
+        return containsNull || !isAllOrNone;
+      case FALSE:
+        // "unknown as false" can not be simplified to
+        // "true" or "IS NULL"
+        return !containsNull || !isAllOrNone;
+      default:
+        return true;
+      }
+    }
+
+    /** Returns whether it is worth to fix and convert to {@code SEARCH} calls. */
+    boolean needToFix(RexUnknownAs unknownAs) {
+      // Fix and converts to SEARCH if:
+      // 1. A Sarg has complexity greater than 1;
+      // 2. The terms are reduced as simpler Sarg points;
+      // 3. The terms are reduced as simpler Sarg comparison.
+
+      // Ignore 'negate' just to be compatible with previous versions of this
+      // method. "build().complexity()" would be a better estimate, if we could
+      // switch to it breaking lots of plans.
+      final Collection<RexSargBuilder> builders = map.values();
+      if (builders.stream().anyMatch(b -> b.build(false).complexity() > 1)) {
+        return true;
+      }
+      if (builders.size() == 1
+          && !canMerge(builders.iterator().next().build(), unknownAs)) {
+        return false;
+      }
+      return newTermsCount == 1
+          && map.values().stream().allMatch(b -> simpleSarg(b.build()));
+    }
+
+    /**
+     * Returns whether this Sarg can be expanded to more simple form, e.g.
+     * the IN call or single comparison.
+     */
+    private static boolean simpleSarg(Sarg sarg) {
+      return sarg.isPoints() || RangeSets.isOpenInterval(sarg.rangeSet);
+    }
+
     /** If a term is a call to {@code SEARCH} on a {@link RexSargBuilder},
      * converts it to a {@code SEARCH} on a {@link Sarg}. */
     RexNode fix(RexBuilder rexBuilder, RexNode term) {
       if (term instanceof RexSargBuilder) {
-        RexSargBuilder sargBuilder = (RexSargBuilder) term;
+        final RexSargBuilder sargBuilder = (RexSargBuilder) term;
+        final Sarg sarg = sargBuilder.build();
+        if (sarg.complexity() <= 1 && simpleSarg(sarg)) {
+          // Expand small sargs into comparisons in order to avoid plan changes
+          // and better readability.
+          return RexUtil.sargRef(rexBuilder, sargBuilder.ref, sarg, term.getType());
+        }
         return rexBuilder.makeCall(SqlStdOperatorTable.SEARCH, sargBuilder.ref,
-            rexBuilder.makeSearchArgumentLiteral(sargBuilder.build(),
-                term.getType()));
+            rexBuilder.makeSearchArgumentLiteral(sarg, term.getType()));
       }
       return term;
     }
@@ -2751,17 +2820,6 @@ public class RexSimplify {
           + " terms of which " + nullTermCount + " allow null)";
     }
 
-    /** Returns a rough estimate of whether it is worth converting to a Sarg.
-     *
-     * @see Sarg#complexity()
-     */
-    int complexity() {
-      // Ignore 'negate' just to be compatible with previous versions of this
-      // method. "build().complexity()" would be a better estimate, if we could
-      // switch to it breaking lots of plans.
-      return build(false).complexity();
-    }
-
     <C extends Comparable<C>> Sarg<C> build() {
       return build(negate);
     }
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 91ee9f3..53c12ed 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -565,10 +565,17 @@ public class RexUtil {
     };
   }
 
-  /** Expands calls to {@link SqlStdOperatorTable#SEARCH} in an expression. */
+  /** Expands all the calls to {@link SqlStdOperatorTable#SEARCH} in an expression. */
   public static RexNode expandSearch(RexBuilder rexBuilder,
       @Nullable RexProgram program, RexNode node) {
-    return node.accept(searchShuttle(rexBuilder, program, -1));
+    return expandSearch(rexBuilder, program, node, -1);
+  }
+
+  /** Expands calls to {@link SqlStdOperatorTable#SEARCH}
+   * whose complexity is greater than {@code maxComplexity} in an expression. */
+  public static RexNode expandSearch(RexBuilder rexBuilder,
+       @Nullable RexProgram program, RexNode node, int maxComplexity) {
+    return node.accept(searchShuttle(rexBuilder, program, maxComplexity));
   }
 
   /** Creates a shuttle that expands calls to
@@ -585,7 +592,7 @@ public class RexUtil {
   }
 
   @SuppressWarnings("BetaApi")
-  private static <C extends Comparable<C>> RexNode sargRef(
+  public static <C extends Comparable<C>> RexNode sargRef(
       RexBuilder rexBuilder, RexNode ref, Sarg<C> sarg, RelDataType type) {
     if (sarg.isAll()) {
       if (sarg.containsNull) {
@@ -3010,7 +3017,24 @@ public class RexUtil {
     }
 
     @Override public RexNode visitCall(RexCall call) {
+      final boolean[] update = {false};
+      final List<RexNode> clonedOperands;
       switch (call.getKind()) {
+      // Flatten AND/OR operands.
+      case OR:
+        clonedOperands = visitList(call.operands, update);
+        if (update[0]) {
+          return composeDisjunction(rexBuilder, clonedOperands);
+        } else {
+          return call;
+        }
+      case AND:
+        clonedOperands = visitList(call.operands, update);
+        if (update[0]) {
+          return composeConjunction(rexBuilder, clonedOperands);
+        } else {
+          return call;
+        }
       case SEARCH:
         final RexNode ref = call.operands.get(0);
         final RexLiteral literal =
diff --git a/core/src/main/java/org/apache/calcite/util/RangeSets.java b/core/src/main/java/org/apache/calcite/util/RangeSets.java
index c68803f..70b8168 100644
--- a/core/src/main/java/org/apache/calcite/util/RangeSets.java
+++ b/core/src/main/java/org/apache/calcite/util/RangeSets.java
@@ -23,6 +23,7 @@ import com.google.common.collect.RangeSet;
 import com.google.common.collect.TreeRangeSet;
 
 import java.util.Iterator;
+import java.util.Set;
 import java.util.function.BiConsumer;
 import java.util.function.Function;
 
@@ -128,6 +129,17 @@ public class RangeSets {
         && !range.isEmpty();
   }
 
+  /** Returns whether a range set is a single open interval. */
+  public static <C extends Comparable<C>> boolean isOpenInterval(RangeSet<C> rangeSet) {
+    if (rangeSet.isEmpty()) {
+      return false;
+    }
+    final Set<Range<C>> ranges = rangeSet.asRanges();
+    final Range<C> range = ranges.iterator().next();
+    return ranges.size() == 1
+        && (!range.hasLowerBound() || !range.hasUpperBound());
+  }
+
   /** Returns the number of ranges in a range set that are points.
    *
    * <p>If every range in a range set is a point then it can be converted to a
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 ba2f91b..e68d0a2 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
@@ -936,9 +936,7 @@ class RexProgramTest extends RexProgramTestBase {
         "AND(<=(?0.h, 1), >(?0.h, 1))",
         "false");
 
-    checkSimplify2(and(le(hRef, literal(1)), ge(hRef, literal(1))),
-        "AND(<=(?0.h, 1), >=(?0.h, 1))",
-        "=(?0.h, 1)");
+    checkSimplify(and(le(hRef, literal(1)), ge(hRef, literal(1))), "=(?0.h, 1)");
 
     checkSimplify2(and(lt(hRef, literal(1)), eq(hRef, literal(1)), ge(hRef, literal(1))),
         "AND(<(?0.h, 1), =(?0.h, 1), >=(?0.h, 1))",
@@ -984,22 +982,22 @@ class RexProgramTest extends RexProgramTestBase {
     // "x <> x" simplifies to "false"
     checkSimplify(ne(literal(1), literal(1)), "false");
     checkSimplify(ne(hRef, hRef), "false");
-    checkSimplify3(ne(iRef, iRef), "AND(null, SEARCH(?0.i, Sarg[NULL]))",
-        "false", "SEARCH(?0.i, Sarg[NULL])");
+    checkSimplify3(ne(iRef, iRef), "AND(null, IS NULL(?0.i))",
+        "false", "IS NULL(?0.i)");
     checkSimplifyUnchanged(ne(iRef, hRef));
 
     // "x < x" simplifies to "false"
     checkSimplify(lt(literal(1), literal(1)), "false");
     checkSimplify(lt(hRef, hRef), "false");
-    checkSimplify3(lt(iRef, iRef), "AND(null, SEARCH(?0.i, Sarg[NULL]))",
-        "false", "SEARCH(?0.i, Sarg[NULL])");
+    checkSimplify3(lt(iRef, iRef), "AND(null, IS NULL(?0.i))",
+        "false", "IS NULL(?0.i)");
     checkSimplifyUnchanged(lt(iRef, hRef));
 
     // "x > x" simplifies to "false"
     checkSimplify(gt(literal(1), literal(1)), "false");
     checkSimplify(gt(hRef, hRef), "false");
-    checkSimplify3(gt(iRef, iRef), "AND(null, SEARCH(?0.i, Sarg[NULL]))",
-        "false", "SEARCH(?0.i, Sarg[NULL])");
+    checkSimplify3(gt(iRef, iRef), "AND(null, IS NULL(?0.i))",
+        "false", "IS NULL(?0.i)");
     checkSimplifyUnchanged(gt(iRef, hRef));
 
     // "(not x) is null" to "x is null"
@@ -1133,7 +1131,7 @@ class RexProgramTest extends RexProgramTestBase {
     checkSimplifyFilter(
         and(lt(literal(1), aRef), lt(literal(5), aRef)),
         RelOptPredicateList.EMPTY,
-        "<(5, ?0.a)");
+        ">(?0.a, 5)");
 
     // condition "1 < a && a < 5" is converted to a Sarg
     checkSimplifyFilter(
@@ -1145,7 +1143,7 @@ class RexProgramTest extends RexProgramTestBase {
     checkSimplifyFilter(
         and(gt(literal(1), aRef), gt(literal(5), aRef)),
         RelOptPredicateList.EMPTY,
-        ">(1, ?0.a)");
+        "<(?0.a, 1)");
 
     // condition "1 > a && a > 5" yields false
     checkSimplifyFilter(
@@ -1268,7 +1266,9 @@ class RexProgramTest extends RexProgramTestBase {
         or(
             ne(vInt(), literal(1)),
             ne(vInt(), literal(2))),
-        "OR(IS NOT NULL(?0.int0), null)", "IS NOT NULL(?0.int0)", "true");
+        "OR(IS NOT NULL(?0.int0), null)",
+        "IS NOT NULL(?0.int0)",
+        "true");
   }
 
   /** Test case for
@@ -1475,11 +1475,11 @@ class RexProgramTest extends RexProgramTestBase {
             ne(aRef, literal4)),
         "<>(?0.a, 4)");
 
-    // "b <> 1 or b = 1" cannot be simplified, because b might be null
+    // "b <> 1 or b = 1" ==> "b is not null" with unknown as false
     final RexNode neOrEq =
         or(ne(bRef, literal(1)),
             eq(bRef, literal(1)));
-    checkSimplifyFilter(neOrEq, "OR(<>(?0.b, 1), =(?0.b, 1))");
+    checkSimplifyFilter(neOrEq, "IS NOT NULL(?0.b)");
 
     // Careful of the excluded middle!
     // We cannot simplify "b <> 1 or b = 1" to "true" because if b is null, the
@@ -1655,8 +1655,8 @@ class RexProgramTest extends RexProgramTestBase {
     // a in (1, 2) or b is null
     RexNode expr = or(eq(aRef, literal(1)), eq(aRef, literal(2)), isNull(bRef));
     final String simplified =
-        "OR(SEARCH($1, Sarg[NULL]), SEARCH($0, Sarg[1, 2]))";
-    final String expanded = "OR(IS NULL($1), OR(=($0, 1), =($0, 2)))";
+        "OR(IS NULL($1), SEARCH($0, Sarg[1, 2]))";
+    final String expanded = "OR(IS NULL($1), =($0, 1), =($0, 2))";
     checkSimplify(expr, simplified)
         .expandedSearch(expanded);
   }
@@ -1687,10 +1687,8 @@ class RexProgramTest extends RexProgramTestBase {
         isNotNull(bRef));
     // [CALCITE-4352] causes "and b is not null" to disappear from the expanded
     // form.
-    final String simplified = "AND(SEARCH($0, Sarg[(0..10)]),"
-        + " SEARCH($1, Sarg[NOT NULL]))";
-    final String expanded =
-        "AND(AND(>($0, 0), <($0, 10)), IS NOT NULL($1))";
+    final String simplified = "AND(SEARCH($0, Sarg[(0..10)]), IS NOT NULL($1))";
+    final String expanded = "AND(>($0, 0), <($0, 10), IS NOT NULL($1))";
     checkSimplify(expr, simplified)
         .expandedSearch(expanded);
   }
@@ -1705,9 +1703,8 @@ class RexProgramTest extends RexProgramTestBase {
         isNull(bRef));
     // [CALCITE-4352] causes "and b is null" to disappear from the expanded
     // form.
-    final String simplified =
-        "AND(SEARCH($0, Sarg[(0..10)]), SEARCH($1, Sarg[NULL]))";
-    final String expanded = "AND(AND(>($0, 0), <($0, 10)), IS NULL($1))";
+    final String simplified = "AND(SEARCH($0, Sarg[(0..10)]), IS NULL($1))";
+    final String expanded = "AND(>($0, 0), <($0, 10), IS NULL($1))";
     checkSimplify(expr, simplified)
         .expandedSearch(expanded);
   }
@@ -1747,7 +1744,7 @@ class RexProgramTest extends RexProgramTestBase {
             or(ne(vInt(), literal(20)),
                 isNull(vInt())),
         eq(vInt(), literal(10)));
-    checkSimplify2(e, "SEARCH(?0.int0, Sarg[10])", "=(?0.int0, 10)");
+    checkSimplify(e, "=(?0.int0, 10)");
   }
 
   @Test void testSimplifyEqOrIsNullAndEq() {
@@ -1771,7 +1768,65 @@ class RexProgramTest extends RexProgramTestBase {
             or(eq(vInt(), literal(10)),
                 isNull(vInt())),
         eq(vInt(), literal(10)));
-    checkSimplify2(e, "SEARCH(?0.int0, Sarg[10])", "=(?0.int0, 10)");
+    checkSimplify(e, "=(?0.int0, 10)");
+  }
+
+  @Test void testSimplifyInAnd() {
+    // deptno in (20, 10) and deptno = 10
+    //   ==>
+    // deptno = 10
+    checkSimplify(
+        and(
+            in(vInt(), literal(20), literal(10)),
+            eq(vInt(), literal(10))),
+        "=(?0.int0, 10)");
+
+    // deptno in (20, 10) and deptno = 30
+    //   ==>
+    // false
+    checkSimplify2(
+        and(
+        in(vInt(), literal(20), literal(10)),
+        eq(vInt(), literal(30))),
+        "AND(SEARCH(?0.int0, Sarg[10, 20]), =(?0.int0, 30))",
+        "false");
+  }
+
+  @Test void testSimplifyInOr() {
+    // deptno > 0 or deptno in (20, 10)
+    //   ==>
+    // deptno > 0
+    checkSimplify(
+        or(
+            gt(vInt(), literal(0)),
+            in(vInt(), literal(20), literal(10))),
+        ">(?0.int0, 0)");
+  }
+
+  /** Test strategies for {@code SargCollector.canMerge(Sarg, RexUnknownAs)}. */
+  @Test void testSargMerge() {
+    checkSimplify2(
+        or(
+            ne(vInt(), literal(1)),
+            eq(vInt(), literal(1))),
+        "OR(<>(?0.int0, 1), =(?0.int0, 1))",
+        "IS NOT NULL(?0.int0)");
+    checkSimplify2(
+        and(
+            gt(vInt(), literal(5)),
+            lt(vInt(), literal(3))),
+        "AND(>(?0.int0, 5), <(?0.int0, 3))",
+        "false");
+    checkSimplify(
+        or(
+            falseLiteral,
+            isNull(vInt())),
+        "IS NULL(?0.int0)");
+    checkSimplify(
+        and(
+            trueLiteral,
+            isNotNull(vInt())),
+        "IS NOT NULL(?0.int0)");
   }
 
   @Test void testSimplifyUnknown() {
@@ -1838,7 +1893,7 @@ class RexProgramTest extends RexProgramTestBase {
   @Test void testNestedAndSimplification() {
     // to have the correct mode for the AND at the bottom,
     // both the OR and AND parent should retain the UnknownAs mode
-    checkSimplify2(
+    checkSimplify(
         and(
             eq(vInt(2), literal(2)),
             or(
@@ -1846,7 +1901,6 @@ class RexProgramTest extends RexProgramTestBase {
                 and(
                     ge(vInt(), literal(1)),
                     le(vInt(), literal(1))))),
-        "AND(=(?0.int2, 2), OR(=(?0.int3, 3), AND(>=(?0.int0, 1), <=(?0.int0, 1))))",
         "AND(=(?0.int2, 2), OR(=(?0.int3, 3), =(?0.int0, 1)))");
   }
 
@@ -2406,9 +2460,9 @@ class RexProgramTest extends RexProgramTestBase {
         "AND(IS NOT NULL(?0.int0), IS NOT NULL(?0.int1))",
         "true");
     checkSimplify3(and(ne(vInt(), vInt()), ne(vInt(1), vInt(1))),
-        "AND(null, SEARCH(?0.int0, Sarg[NULL]), SEARCH(?0.int1, Sarg[NULL]))",
+        "AND(null, IS NULL(?0.int0), IS NULL(?0.int1))",
         "false",
-        "AND(SEARCH(?0.int0, Sarg[NULL]), SEARCH(?0.int1, Sarg[NULL]))");
+        "AND(IS NULL(?0.int0), IS NULL(?0.int1))");
   }
 
   @Test void testBooleanComparisons() {
@@ -2700,17 +2754,15 @@ class RexProgramTest extends RexProgramTestBase {
     //    -> "x = x AND y < y" (treating unknown as unknown)
     //    -> false (treating unknown as false)
     checkSimplify3(and(eq(vInt(1), vInt(1)), not(ge(vInt(2), vInt(2)))),
-        "AND(OR(null, IS NOT NULL(?0.int1)), null,"
-            + " SEARCH(?0.int2, Sarg[NULL]))",
+        "AND(OR(null, IS NOT NULL(?0.int1)), null, IS NULL(?0.int2))",
         "false",
-        "SEARCH(?0.int2, Sarg[NULL])");
+        "IS NULL(?0.int2)");
 
     // "NOT(x = x AND NOT (y = y))"
     //   -> "OR(x <> x, y >= y)" (treating unknown as unknown)
     //   -> "y IS NOT NULL" (treating unknown as false)
     checkSimplify3(not(and(eq(vInt(1), vInt(1)), not(ge(vInt(2), vInt(2))))),
-        "OR(AND(null, SEARCH(?0.int1, Sarg[NULL])), null,"
-            + " IS NOT NULL(?0.int2))",
+        "OR(AND(null, IS NULL(?0.int1)), null, IS NOT NULL(?0.int2))",
         "IS NOT NULL(?0.int2)",
         "true");
   }
@@ -2733,8 +2785,7 @@ class RexProgramTest extends RexProgramTestBase {
     //    -> "x = x OR y < y" (treating unknown as unknown)
     //    -> "x IS NOT NULL" (treating unknown as false)
     checkSimplify3(or(eq(vInt(1), vInt(1)), not(ge(vInt(2), vInt(2)))),
-        "OR(null, IS NOT NULL(?0.int1),"
-            + " AND(null, SEARCH(?0.int2, Sarg[NULL])))",
+        "OR(null, IS NOT NULL(?0.int1), AND(null, IS NULL(?0.int2)))",
         "IS NOT NULL(?0.int1)",
         "true");
 
@@ -2742,10 +2793,9 @@ class RexProgramTest extends RexProgramTestBase {
     //   -> "AND(x <> x, y >= y)" (treating unknown as unknown)
     //   -> "FALSE" (treating unknown as false)
     checkSimplify3(not(or(eq(vInt(1), vInt(1)), not(ge(vInt(2), vInt(2))))),
-        "AND(null, SEARCH(?0.int1, Sarg[NULL]),"
-            + " OR(null, IS NOT NULL(?0.int2)))",
+        "AND(null, IS NULL(?0.int1), OR(null, IS NOT NULL(?0.int2)))",
         "false",
-        "SEARCH(?0.int1, Sarg[NULL])");
+        "IS NULL(?0.int1)");
   }
 
   private void checkSarg(String message, Sarg sarg,
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 be5d678..e4b185f 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -3348,8 +3348,7 @@ public class RelBuilderTest {
                         b.literal(11), b.literal(10))))
             .build();
     final String expected = ""
-        + "LogicalFilter(condition=[OR(SEARCH($7, Sarg[10, 11, (15..+∞)]), "
-        + "SEARCH($2, Sarg['CLERK']:CHAR(5)))])\n"
+        + "LogicalFilter(condition=[OR(SEARCH($7, Sarg[10, 11, (15..+∞)]), =($2, 'CLERK'))])\n"
         + "  LogicalTableScan(table=[[scott, EMP]])\n";
     assertThat(f.apply(createBuilder()), hasTree(expected));
     assertThat(f.apply(createBuilder(c -> c.withSimplify(false))),
@@ -3773,8 +3772,7 @@ public class RelBuilderTest {
             builder.equals(builder.field("DEPTNO"), builder.literal(10)));
     final RelNode root3 = builder.build();
     final String expectedRel3 = ""
-        + "LogicalFilter(condition=[AND(SEARCH($0, Sarg[[1..3), (3..5]]), "
-        + "SEARCH($7, Sarg[10]))])\n"
+        + "LogicalFilter(condition=[AND(SEARCH($0, Sarg[[1..3), (3..5]]), =($7, 10))])\n"
         + "  LogicalTableScan(table=[[scott, EMP]])\n";
     assertThat(root3, hasTree(expectedRel3));
   }
diff --git a/core/src/test/java/org/apache/calcite/util/RangeSetTest.java b/core/src/test/java/org/apache/calcite/util/RangeSetTest.java
index f048705..904979a 100644
--- a/core/src/test/java/org/apache/calcite/util/RangeSetTest.java
+++ b/core/src/test/java/org/apache/calcite/util/RangeSetTest.java
@@ -100,6 +100,38 @@ class RangeSetTest {
     assertThat(RangeSets.isPoint(Range.atLeast(0)), is(false));
   }
 
+  /** Tests {@link RangeSets#isOpenInterval(RangeSet)}. */
+  @Test void testRangeSetIsOpenInterval() {
+    final RangeSet<Integer> setGt0 = ImmutableRangeSet.of(Range.greaterThan(0));
+    final RangeSet<Integer> setAl0 = ImmutableRangeSet.of(Range.atLeast(0));
+    final RangeSet<Integer> setLt0 = ImmutableRangeSet.of(Range.lessThan(0));
+    final RangeSet<Integer> setAm0 = ImmutableRangeSet.of(Range.atMost(0));
+
+    assertThat(RangeSets.isOpenInterval(setGt0), is(true));
+    assertThat(RangeSets.isOpenInterval(setAl0), is(true));
+    assertThat(RangeSets.isOpenInterval(setLt0), is(true));
+    assertThat(RangeSets.isOpenInterval(setAm0), is(true));
+
+    final RangeSet<Integer> setNone = ImmutableRangeSet.of();
+    final RangeSet<Integer> multiRanges = ImmutableRangeSet.<Integer>builder()
+        .add(Range.lessThan(0))
+        .add(Range.greaterThan(3))
+        .build();
+
+    assertThat(RangeSets.isOpenInterval(setNone), is(false));
+    assertThat(RangeSets.isOpenInterval(multiRanges), is(false));
+
+    final RangeSet<Integer> open = ImmutableRangeSet.of(Range.open(0, 3));
+    final RangeSet<Integer> closed = ImmutableRangeSet.of(Range.closed(0, 3));
+    final RangeSet<Integer> openClosed = ImmutableRangeSet.of(Range.openClosed(0, 3));
+    final RangeSet<Integer> closedOpen = ImmutableRangeSet.of(Range.closedOpen(0, 3));
+
+    assertThat(RangeSets.isOpenInterval(open), is(false));
+    assertThat(RangeSets.isOpenInterval(closed), is(false));
+    assertThat(RangeSets.isOpenInterval(openClosed), is(false));
+    assertThat(RangeSets.isOpenInterval(closedOpen), is(false));
+  }
+
   /** Tests {@link RangeSets#countPoints(RangeSet)}. */
   @Test void testRangeCountPoints() {
     final Fixture f = new Fixture();
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 e78a0dc..e9dd972 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -6163,7 +6163,7 @@ where d.deptno in (7, 9) or d.comm > 10]]>
             <![CDATA[
 LogicalProject(EXPR$0=[1])
   LogicalJoin(condition=[=($7, $16)], joinType=[inner])
-    LogicalFilter(condition=[OR(SEARCH($7, Sarg[7, 9]), SEARCH($6, Sarg[(10..+∞)]))])
+    LogicalFilter(condition=[OR(SEARCH($7, Sarg[7, 9]), >($6, 10))])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -6172,7 +6172,7 @@ LogicalProject(EXPR$0=[1])
             <![CDATA[
 LogicalProject(EXPR$0=[1])
   LogicalJoin(condition=[=($7, $16)], joinType=[inner])
-    LogicalFilter(condition=[OR(=($7, 7), =($7, 9), >($6, 10))])
+    LogicalFilter(condition=[OR(SEARCH($7, Sarg[7, 9]), >($6, 10))])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -6221,7 +6221,7 @@ LogicalProject(EXPR$0=[1])
       LogicalProject(DEPTNO=[$7])
         LogicalFilter(condition=[>($7, 1)])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-    LogicalFilter(condition=[OR(>($7, 7), >($7, 1))])
+    LogicalFilter(condition=[>($7, 1)])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -11184,7 +11184,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EMPNO=[10], ENAME=[$1], JOB=[$2], MGR=[null:INTEGER], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[7], SLACKER=[$8])
-  LogicalFilter(condition=[AND(SEARCH($7, Sarg[7]), SEARCH($0, Sarg[10]), SEARCH($3, Sarg[NULL]))])
+  LogicalFilter(condition=[AND(=($7, 7), =($0, 10), IS NULL($3))])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -11718,7 +11718,7 @@ LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null:INTEGER)])
             <![CDATA[
 LogicalProject(EMPNO=[$0])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    LogicalFilter(condition=[<($0, CASE(OR(AND(IS NOT NULL($12), <>($9, 0)), AND(<($10, $9), null, SEARCH($9, Sarg[(-∞..0), (0..+∞)]), SEARCH($12, Sarg[NULL]))), 10, AND(OR(IS NULL($12), =($9, 0)), OR(>=($10, $9), =($9, 0), SEARCH($12, Sarg[NOT NULL]))), 20, 30))])
+    LogicalFilter(condition=[<($0, CASE(OR(AND(IS NOT NULL($12), <>($9, 0)), AND(<($10, $9), null, SEARCH($9, Sarg[(-∞..0), (0..+∞)]), IS NULL($12))), 10, AND(OR(IS NULL($12), =($9, 0)), OR(>=($10, $9), =($9, 0), IS NOT NULL($12))), 20, 30))])
       LogicalJoin(condition=[=($7, $11)], joinType=[left])
         LogicalJoin(condition=[true], joinType=[inner])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
diff --git a/core/src/test/resources/sql/sub-query.iq b/core/src/test/resources/sql/sub-query.iq
index e5cd646..a7fc8a3 100644
--- a/core/src/test/resources/sql/sub-query.iq
+++ b/core/src/test/resources/sql/sub-query.iq
@@ -32,7 +32,7 @@ where t1.x not in (select t2.x from t2);
 (0 rows)
 
 !ok
-EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t1, $t5)], expr#7=[Sarg[NOT NULL]], expr#8=[SEARCH($t0, $t7)], expr#9=[Sarg[NULL]], expr#10=[SEARCH($t4, $t9)], expr#11=[>=($t2, $t1)], expr#12=[AND($t8, $t10, $t11)], expr#13=[OR($t6, $t12)], X=[$t0], $condition=[$t13])
+EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t1, $t5)], expr#7=[IS NULL($t4)], expr#8=[>=($t2, $t1)], expr#9=[IS NOT NULL($t0)], expr#10=[AND($t7, $t8, $t9)], expr#11=[OR($t6, $t10)], X=[$t0], $condition=[$t11])
   EnumerableHashJoin(condition=[=($0, $3)], joinType=[left])
     EnumerableNestedLoopJoin(condition=[true], joinType=[inner])
       EnumerableUnion(all=[true])
diff --git a/druid/src/test/java/org/apache/calcite/test/DruidAdapter2IT.java b/druid/src/test/java/org/apache/calcite/test/DruidAdapter2IT.java
index 0cd869d..2ea1bef 100644
--- a/druid/src/test/java/org/apache/calcite/test/DruidAdapter2IT.java
+++ b/druid/src/test/java/org/apache/calcite/test/DruidAdapter2IT.java
@@ -1027,9 +1027,9 @@ public class DruidAdapter2IT {
         + "  DruidQuery(table=[[foodmart, foodmart]], "
         + "intervals=[[1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z]], "
         + "filter=[AND("
-        + "SEARCH($3, Sarg['High Top Dried Mushrooms':VARCHAR]:VARCHAR), "
+        + "=($3, 'High Top Dried Mushrooms'), "
         + "SEARCH($87, Sarg['Q2', 'Q3']:CHAR(2)), "
-        + "SEARCH($30, Sarg['WA':VARCHAR]:VARCHAR))], "
+        + "=($30, 'WA'))], "
         + "projects=[[$30, $29, $3]], groups=[{0, 1, 2}], aggs=[[]])\n";
     sql(sql)
         .queryContains(new DruidChecker(druidQuery1, druidQuery2))
@@ -1069,9 +1069,9 @@ public class DruidAdapter2IT {
         + "  DruidQuery(table=[[foodmart, foodmart]], "
         + "intervals=[[1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z]], "
         + "filter=[AND("
-        + "SEARCH($3, Sarg['High Top Dried Mushrooms':VARCHAR]:VARCHAR), "
+        + "=($3, 'High Top Dried Mushrooms'), "
         + "SEARCH($87, Sarg['Q2', 'Q3']:CHAR(2)), "
-        + "SEARCH($30, Sarg['WA':VARCHAR]:VARCHAR))], "
+        + "=($30, 'WA'))], "
         + "projects=[[$30, $29, $3]])\n";
     sql(sql)
         .queryContains(new DruidChecker(druidQuery))
diff --git a/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java b/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
index b88420e..8fbb099 100644
--- a/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
+++ b/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
@@ -1295,9 +1295,9 @@ public class DruidAdapterIT {
         + "  DruidQuery(table=[[foodmart, foodmart]], "
         + "intervals=[[1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z]], "
         + "filter=[AND("
-        + "SEARCH($3, Sarg['High Top Dried Mushrooms':VARCHAR]:VARCHAR), "
+        + "=($3, 'High Top Dried Mushrooms'), "
         + "SEARCH($87, Sarg['Q2', 'Q3']:CHAR(2)), "
-        + "SEARCH($30, Sarg['WA':VARCHAR]:VARCHAR))], "
+        + "=($30, 'WA'))], "
         + "projects=[[$30, $29, $3]], groups=[{0, 1, 2}], aggs=[[]])\n";
     sql(sql)
         .queryContains(new DruidChecker(druidQuery1, druidQuery2))
@@ -1337,9 +1337,9 @@ public class DruidAdapterIT {
         + "  DruidQuery(table=[[foodmart, foodmart]], "
         + "intervals=[[1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z]], "
         + "filter=[AND("
-        + "SEARCH($3, Sarg['High Top Dried Mushrooms':VARCHAR]:VARCHAR), "
+        + "=($3, 'High Top Dried Mushrooms'), "
         + "SEARCH($87, Sarg['Q2', 'Q3']:CHAR(2)), "
-        + "SEARCH($30, Sarg['WA':VARCHAR]:VARCHAR))], "
+        + "=($30, 'WA'))], "
         + "projects=[[$30, $29, $3]])\n";
     sql(sql)
         .queryContains(new DruidChecker(druidQuery))