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 2020/11/01 17:46:19 UTC

[calcite] 04/04: [CALCITE-4352] RexSimplify incorrectly drops IS NULL and IS NOT NULL from SEARCH expressions

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

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

commit d03ce47536ddb2aaec6d445b3ea48464081a7ed0
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Sat Oct 24 17:23:16 2020 -0700

    [CALCITE-4352] RexSimplify incorrectly drops IS NULL and IS NOT NULL from SEARCH expressions
    
    Improve Sarg.toString() for IS NULL, IS NOT NULL, TRUE,
    FALSE.
    
    Simplify NOT SEARCH(x, sarg) to SEARCH(x, sarg2) where sarg2
    is the complement of sarg.
    
    Improve return type of SEARCH; result is NOT NULL if argument
    is NOT NULL of it Sarg allows IS NULL.
    
    Close apache/calcite#2233
---
 .../main/java/org/apache/calcite/rex/RexCall.java  |   9 ++
 .../java/org/apache/calcite/rex/RexSimplify.java   |  19 ++-
 .../main/java/org/apache/calcite/rex/RexUtil.java  |  84 ++++++++---
 .../main/java/org/apache/calcite/sql/SqlKind.java  |  11 ++
 .../apache/calcite/sql/fun/SqlSearchOperator.java  |  26 +++-
 .../java/org/apache/calcite/tools/RelBuilder.java  |   7 +-
 .../main/java/org/apache/calcite/util/Sarg.java    |  28 +++-
 .../org/apache/calcite/rex/RexProgramTest.java     | 168 ++++++++++++++-------
 .../org/apache/calcite/test/RelBuilderTest.java    |   4 +-
 .../org/apache/calcite/test/RelOptRulesTest.xml    |  10 +-
 core/src/test/resources/sql/conditions.iq          |  27 ++++
 core/src/test/resources/sql/sub-query.iq           |   2 +-
 12 files changed, 302 insertions(+), 93 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rex/RexCall.java b/core/src/main/java/org/apache/calcite/rex/RexCall.java
index 5bea803..73795c0 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCall.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCall.java
@@ -25,6 +25,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Sarg;
 
 import com.google.common.collect.ImmutableList;
 
@@ -196,6 +197,10 @@ public class RexCall extends RexNode {
     case IS_TRUE:
     case CAST:
       return operands.get(0).isAlwaysTrue();
+    case SEARCH:
+      final Sarg sarg = ((RexLiteral) operands.get(1)).getValueAs(Sarg.class);
+      return sarg.isAll()
+          && (sarg.containsNull || !operands.get(0).getType().isNullable());
     default:
       return false;
     }
@@ -213,6 +218,10 @@ public class RexCall extends RexNode {
     case IS_TRUE:
     case CAST:
       return operands.get(0).isAlwaysFalse();
+    case SEARCH:
+      final Sarg sarg = ((RexLiteral) operands.get(1)).getValueAs(Sarg.class);
+      return sarg.isNone()
+          && (!sarg.containsNull || !operands.get(0).getType().isNullable());
     default:
       return false;
     }
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 9e2ab58..65b3ba3 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -583,6 +583,18 @@ public class RexSimplify {
     case NOT:
       // NOT NOT x ==> x
       return simplify(((RexCall) a).getOperands().get(0), unknownAs);
+    case SEARCH:
+      // NOT SEARCH(x, Sarg[(-inf, 10) OR NULL) ==> SEARCH(x, Sarg[[10, +inf)])
+      final RexCall call2 = (RexCall) a;
+      final RexNode ref = call2.operands.get(0);
+      final RexLiteral literal = (RexLiteral) call2.operands.get(1);
+      final Sarg sarg = literal.getValueAs(Sarg.class);
+      return simplifySearch(
+          call2.clone(call2.type,
+              ImmutableList.of(ref,
+                  rexBuilder.makeLiteral(sarg.negate(), literal.getType(),
+                      literal.getTypeName()))),
+          unknownAs.negate());
     case LITERAL:
       if (a.getType().getSqlTypeName() == SqlTypeName.BOOLEAN
           && !RexLiteral.isNullLiteral(a)) {
@@ -2651,7 +2663,7 @@ public class RexSimplify {
           map.computeIfAbsent(e, e2 ->
               addFluent(newTerms, new RexSargBuilder(e2, rexBuilder, negate)));
       if (negate) {
-        kind = kind.negateNullSafe();
+        kind = kind.negateNullSafe2();
       }
       final Comparable value = literal.getValueAs(Comparable.class);
       switch (kind) {
@@ -2687,6 +2699,7 @@ public class RexSimplify {
         } else {
           ++b.notNullTermCount;
         }
+        b.addAll();
         return true;
       case SEARCH:
         final Sarg sarg = literal.getValueAs(Sarg.class);
@@ -2790,6 +2803,10 @@ public class RexSimplify {
       throw new UnsupportedOperationException();
     }
 
+    void addAll() {
+      rangeSet.add(Range.all());
+    }
+
     void addRange(Range<Comparable> range, RelDataType type) {
       types.add(type);
       rangeSet.add(range);
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 bdda82b..91ee9f3 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -565,32 +565,36 @@ public class RexUtil {
     };
   }
 
+  /** Expands calls to {@link SqlStdOperatorTable#SEARCH} in an expression. */
   public static RexNode expandSearch(RexBuilder rexBuilder,
       @Nullable RexProgram program, RexNode node) {
-    final RexShuttle shuttle = new RexShuttle() {
-      @Override public RexNode visitCall(RexCall call) {
-        switch (call.getKind()) {
-        case SEARCH:
-          return visitSearch(rexBuilder, program, call);
-        default:
-          return super.visitCall(call);
-        }
-      }
-    };
-    return node.accept(shuttle);
+    return node.accept(searchShuttle(rexBuilder, program, -1));
+  }
+
+  /** Creates a shuttle that expands calls to
+   * {@link SqlStdOperatorTable#SEARCH}.
+   *
+   * <p>If {@code maxComplexity} is non-negative, a {@link Sarg} whose
+   * complexity is greater than {@code maxComplexity} is retained (not
+   * expanded); this gives a means to simplify simple expressions such as
+   * {@code x IS NULL} or {@code x > 10} while keeping more complex expressions
+   * such as {@code x IN (3, 5, 7) OR x IS NULL} as a Sarg. */
+  public static RexShuttle searchShuttle(RexBuilder rexBuilder,
+      RexProgram program, int maxComplexity) {
+    return new SearchExpandingShuttle(program, rexBuilder, maxComplexity);
   }
 
   @SuppressWarnings("BetaApi")
-  private static <C extends Comparable<C>> RexNode
-      visitSearch(RexBuilder rexBuilder,
-          @Nullable RexProgram program, RexCall call) {
-    final RexNode ref = call.operands.get(0);
-    final RexLiteral literal =
-        (RexLiteral) deref(program, call.operands.get(1));
-    @SuppressWarnings("unchecked")
-    final Sarg<C> sarg = literal.getValueAs(Sarg.class);
+  private static <C extends Comparable<C>> RexNode sargRef(
+      RexBuilder rexBuilder, RexNode ref, Sarg<C> sarg, RelDataType type) {
+    if (sarg.isAll()) {
+      if (sarg.containsNull) {
+        return rexBuilder.makeLiteral(true);
+      } else {
+        return rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, ref);
+      }
+    }
     final List<RexNode> orList = new ArrayList<>();
-
     if (sarg.containsNull) {
       orList.add(rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, ref));
     }
@@ -600,19 +604,19 @@ public class RexUtil {
           orList.add(
               rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, ref,
                   rexBuilder.makeLiteral(range.lowerEndpoint(),
-                      literal.getType(), true, true))));
+                      type, true, true))));
     } else if (sarg.isComplementedPoints()) {
       // Generate 'ref <> value1 AND ... AND ref <> valueN'
       final List<RexNode> list = sarg.rangeSet.complement().asRanges().stream()
           .map(range ->
               rexBuilder.makeCall(SqlStdOperatorTable.NOT_EQUALS, ref,
                   rexBuilder.makeLiteral(range.lowerEndpoint(),
-                      literal.getType(), true, true)))
+                      type, true, true)))
           .collect(Util.toImmutableList());
       orList.add(composeConjunction(rexBuilder, list));
     } else {
       final RangeSets.Consumer<C> consumer =
-          new RangeToRex<>(ref, orList, rexBuilder, literal.getType());
+          new RangeToRex<>(ref, orList, rexBuilder, type);
       RangeSets.forEach(sarg.rangeSet, consumer);
     }
     return composeDisjunction(rexBuilder, orList);
@@ -2987,4 +2991,38 @@ public class RexUtil {
           op(SqlStdOperatorTable.LESS_THAN, upper));
     }
   }
+
+  /** Shuttle that expands calls to
+   * {@link org.apache.calcite.sql.fun.SqlStdOperatorTable#SEARCH}.
+   *
+   * <p>Calls whose complexity is greater than {@link #maxComplexity}
+   * are retained (not expanded). */
+  private static class SearchExpandingShuttle extends RexShuttle {
+    private final RexBuilder rexBuilder;
+    private final RexProgram program;
+    private final int maxComplexity;
+
+    SearchExpandingShuttle(RexProgram program, RexBuilder rexBuilder,
+        int maxComplexity) {
+      this.program = program;
+      this.rexBuilder = rexBuilder;
+      this.maxComplexity = maxComplexity;
+    }
+
+    @Override public RexNode visitCall(RexCall call) {
+      switch (call.getKind()) {
+      case SEARCH:
+        final RexNode ref = call.operands.get(0);
+        final RexLiteral literal =
+            (RexLiteral) deref(program, call.operands.get(1));
+        final Sarg sarg = literal.getValueAs(Sarg.class);
+        if (maxComplexity < 0 || sarg.complexity() < maxComplexity) {
+          return sargRef(rexBuilder, ref, sarg, literal.getType());
+        }
+        // Sarg is complex (therefore useful); fall through
+      default:
+        return super.visitCall(call);
+      }
+    }
+  }
 }
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 aa1332e..719bbf6 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -1385,6 +1385,17 @@ public enum SqlKind {
     }
   }
 
+  public SqlKind negateNullSafe2() {
+    switch (this) {
+    case IS_NOT_NULL:
+      return IS_NULL;
+    case IS_NULL:
+      return IS_NOT_NULL;
+    default:
+      return this.negateNullSafe();
+    }
+  }
+
   /**
    * Returns whether this {@code SqlKind} belongs to a given category.
    *
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSearchOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSearchOperator.java
index 5aa9f5f..6c5ab79 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSearchOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSearchOperator.java
@@ -16,18 +16,42 @@
  */
 package org.apache.calcite.sql.fun;
 
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlInternalOperator;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
 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.util.Sarg;
 
 /** Operator that tests whether its left operand is included in the range of
  * values covered by search arguments. */
 class SqlSearchOperator extends SqlInternalOperator {
   SqlSearchOperator() {
-    super("SEARCH", SqlKind.SEARCH, 30, true, ReturnTypes.BOOLEAN_NULLABLE,
+    super("SEARCH", SqlKind.SEARCH, 30, true,
+        ReturnTypes.BOOLEAN.andThen(SqlSearchOperator::makeNullable),
         InferTypes.FIRST_KNOWN,
         OperandTypes.COMPARABLE_UNORDERED_COMPARABLE_UNORDERED);
   }
+
+  /** Sets whether a call to SEARCH should allow nulls.
+   *
+   * <p>For example, if the type of {@code x} is NOT NULL, then
+   * {@code SEARCH(x, Sarg[10])} will never return UNKNOWN.
+   * It is evident from the expansion, "x = 10", but holds for all Sarg
+   * values.
+   *
+   * <p>If {@link Sarg#containsNull} is true, SEARCH will never return
+   * UNKNOWN. For example, {@code SEARCH(x, Sarg[10 OR NULL])} expands to
+   * {@code x = 10 OR x IS NOT NULL}, which returns {@code TRUE} if
+   * {@code x} is NULL, {@code TRUE} if {@code x} is 10, and {@code FALSE}
+   * for all other values.
+   */
+  private static RelDataType makeNullable(SqlOperatorBinding binding,
+      RelDataType type) {
+    final boolean nullable = binding.getOperandType(0).isNullable()
+        && !binding.getOperandLiteralValue(1, Sarg.class).containsNull;
+    return binding.getTypeFactory().createTypeWithNullability(type, nullable);
+  }
 }
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 8209e76..11b77d8 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -1460,8 +1460,13 @@ public class RelBuilder {
 
     // Simplify expressions.
     if (config.simplify()) {
+      final RexShuttle shuttle =
+          RexUtil.searchShuttle(getRexBuilder(), null, 2);
       for (int i = 0; i < nodeList.size(); i++) {
-        nodeList.set(i, simplifier.simplifyPreservingType(nodeList.get(i)));
+        final RexNode node0 = nodeList.get(i);
+        final RexNode node1 = simplifier.simplifyPreservingType(node0);
+        final RexNode node2 = node1.accept(shuttle);
+        nodeList.set(i, node2);
       }
     }
 
diff --git a/core/src/main/java/org/apache/calcite/util/Sarg.java b/core/src/main/java/org/apache/calcite/util/Sarg.java
index b356b1e..ee80daa 100644
--- a/core/src/main/java/org/apache/calcite/util/Sarg.java
+++ b/core/src/main/java/org/apache/calcite/util/Sarg.java
@@ -90,7 +90,7 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
    *
    * and also null is printed as
    *
-   * <blockquote>{@code Sarg[7, 9, (10..+∞), null]}</blockquote>
+   * <blockquote>{@code Sarg[7, 9, (10..+∞) OR NULL]}</blockquote>
    */
   @Override public String toString() {
     final StringBuilder sb = new StringBuilder();
@@ -102,6 +102,12 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
    * with each embedded value. */
   public StringBuilder printTo(StringBuilder sb,
       BiConsumer<StringBuilder, C> valuePrinter) {
+    if (isAll()) {
+      return sb.append(containsNull ? "Sarg[TRUE]" : "Sarg[NOT NULL]");
+    }
+    if (isNone()) {
+      return sb.append(containsNull ? "Sarg[NULL]" : "Sarg[FALSE]");
+    }
     sb.append("Sarg[");
     final RangeSets.Consumer<C> printer = RangeSets.printer(sb, valuePrinter);
     Ord.forEach(rangeSet.asRanges(), (r, i) -> {
@@ -111,7 +117,7 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
       RangeSets.forEach(r, printer);
     });
     if (containsNull) {
-      sb.append(", null");
+      sb.append(" OR NULL");
     }
     return sb.append("]");
   }
@@ -131,6 +137,18 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
         && containsNull == ((Sarg) o).containsNull;
   }
 
+  /** Returns whether this Sarg includes all values (including or not including
+   * null). */
+  public boolean isAll() {
+    return rangeSet.equals(RangeSets.rangeSetAll());
+  }
+
+  /** Returns whether this Sarg includes no values (including or not including
+   * null). */
+  public boolean isNone() {
+    return rangeSet.isEmpty();
+  }
+
   /** Returns whether this Sarg is a collection of 1 or more points (and perhaps
    * an {@code IS NULL} if {@link #containsNull}).
    *
@@ -147,6 +165,7 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
    * or {@code ref NOT IN (value1, ...)}. */
   public boolean isComplementedPoints() {
     return rangeSet.span().encloses(Range.all())
+        && !rangeSet.equals(RangeSets.rangeSetAll())
         && rangeSet.complement().asRanges().stream()
             .allMatch(RangeSets::isPoint);
   }
@@ -182,4 +201,9 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
     }
     return complexity;
   }
+
+  /** Returns a Sarg that matches a value if and only this Sarg does not. */
+  public Sarg negate() {
+    return Sarg.of(!containsNull, rangeSet.complement());
+  }
 }
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 e72550a..ba2f91b 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
@@ -52,6 +52,7 @@ import com.google.common.collect.Multimap;
 import com.google.common.collect.Range;
 import com.google.common.collect.RangeSet;
 
+import org.hamcrest.Matcher;
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
@@ -983,19 +984,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, IS NULL(?0.i))", "false", "IS NULL(?0.i)");
+    checkSimplify3(ne(iRef, iRef), "AND(null, SEARCH(?0.i, Sarg[NULL]))",
+        "false", "SEARCH(?0.i, Sarg[NULL])");
     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, IS NULL(?0.i))", "false", "IS NULL(?0.i)");
+    checkSimplify3(lt(iRef, iRef), "AND(null, SEARCH(?0.i, Sarg[NULL]))",
+        "false", "SEARCH(?0.i, Sarg[NULL])");
     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, IS NULL(?0.i))", "false", "IS NULL(?0.i)");
+    checkSimplify3(gt(iRef, iRef), "AND(null, SEARCH(?0.i, Sarg[NULL]))",
+        "false", "SEARCH(?0.i, Sarg[NULL])");
     checkSimplifyUnchanged(gt(iRef, hRef));
 
     // "(not x) is null" to "x is null"
@@ -1555,7 +1559,7 @@ class RexProgramTest extends RexProgramTestBase {
             eq(aRef, literal(3)),
             or(eq(aRef, literal(3)),
                 eq(aRef, literal(4)))),
-        "AND(SEARCH(?0.b, Sarg[2]), SEARCH(?0.a, Sarg[3]))");
+        "AND(=(?0.b, 2), =(?0.a, 3))");
 
     checkSimplify3(
         or(lt(vInt(), nullInt),
@@ -1588,7 +1592,7 @@ class RexProgramTest extends RexProgramTestBase {
     // a is null or a >= 15
     RexNode expr = or(isNull(aRef),
         ge(aRef, literal(15)));
-    checkSimplify(expr, "SEARCH($0, Sarg[[15..+\u221e), null])")
+    checkSimplify(expr, "SEARCH($0, Sarg[[15..+\u221e) OR NULL])")
         .expandedSearch("OR(IS NULL($0), >=($0, 15))");
   }
 
@@ -1607,7 +1611,7 @@ class RexProgramTest extends RexProgramTestBase {
         ge(aRef, literal(15)));
     // [CALCITE-4190] causes "or a >= 15" to disappear from the simplified form.
     final String simplified =
-        "SEARCH($0, Sarg[(0..12), [15..+\u221e), null])";
+        "SEARCH($0, Sarg[(0..12), [15..+\u221e) OR NULL])";
     final String expanded =
         "OR(IS NULL($0), AND(>($0, 0), <($0, 12)), >=($0, 15))";
     checkSimplify(expr, simplified)
@@ -1636,7 +1640,7 @@ class RexProgramTest extends RexProgramTestBase {
                 eq(aRef, literal(5)))),
         isNull(aRef));
     final String simplified =
-        "SEARCH($0, Sarg[(-\u221e..3), (3..5), (5..+\u221e), null])";
+        "SEARCH($0, Sarg[(-\u221e..3), (3..5), (5..+\u221e) OR NULL])";
     final String expanded = "OR(IS NULL($0), AND(<>($0, 3), <>($0, 5)))";
     checkSimplify(expr, simplified)
         .expandedSearch(expanded);
@@ -1651,7 +1655,7 @@ 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]))";
+        "OR(SEARCH($1, Sarg[NULL]), SEARCH($0, Sarg[1, 2]))";
     final String expanded = "OR(IS NULL($1), OR(=($0, 1), =($0, 2)))";
     checkSimplify(expr, simplified)
         .expandedSearch(expanded);
@@ -1670,6 +1674,44 @@ class RexProgramTest extends RexProgramTestBase {
         .expandedSearch(expanded);
   }
 
+  /** Unit test for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-4352">[CALCITE-4352]
+   * OR simplification incorrectly loses term</a>. */
+  @Test void testSimplifyAndIsNotNull() {
+    final RexNode aRef = input(tInt(true), 0);
+    final RexNode bRef = input(tInt(true), 1);
+    // (0 < a and a < 10) and b is not null
+    RexNode expr = and(
+        and(lt(literal(0), aRef),
+            lt(aRef, literal(10))),
+        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))";
+    checkSimplify(expr, simplified)
+        .expandedSearch(expanded);
+  }
+
+  @Test void testSimplifyAndIsNull() {
+    final RexNode aRef = input(tInt(true), 0);
+    final RexNode bRef = input(tInt(true), 1);
+    // (0 < a and a < 10) and b is null
+    RexNode expr = and(
+        and(lt(literal(0), aRef),
+            lt(aRef, literal(10))),
+        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))";
+    checkSimplify(expr, simplified)
+        .expandedSearch(expanded);
+  }
+
   @Test void testSimplifyItemRangeTerms() {
     RexNode item = item(input(tArray(tInt()), 3), literal(1));
     // paranoid validation doesn't support array types, disable it for a moment
@@ -1717,7 +1759,7 @@ class RexProgramTest extends RexProgramTestBase {
             or(eq(vInt(), literal(20)),
                 isNull(vInt())),
         eq(vInt(), literal(10)));
-    checkSimplify2(e, "SEARCH(?0.int0, Sarg[])", "false");
+    checkSimplify(e, "false");
   }
 
   @Test void testSimplifyEqOrIsNullAndEqSame() {
@@ -2364,9 +2406,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, IS NULL(?0.int0), IS NULL(?0.int1))",
+        "AND(null, SEARCH(?0.int0, Sarg[NULL]), SEARCH(?0.int1, Sarg[NULL]))",
         "false",
-        "AND(IS NULL(?0.int0), IS NULL(?0.int1))");
+        "AND(SEARCH(?0.int0, Sarg[NULL]), SEARCH(?0.int1, Sarg[NULL]))");
   }
 
   @Test void testBooleanComparisons() {
@@ -2658,15 +2700,17 @@ 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, IS NULL(?0.int2))",
+        "AND(OR(null, IS NOT NULL(?0.int1)), null,"
+            + " SEARCH(?0.int2, Sarg[NULL]))",
         "false",
-        "IS NULL(?0.int2)");
+        "SEARCH(?0.int2, Sarg[NULL])");
 
     // "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, IS NULL(?0.int1)), null, IS NOT NULL(?0.int2))",
+        "OR(AND(null, SEARCH(?0.int1, Sarg[NULL])), null,"
+            + " IS NOT NULL(?0.int2))",
         "IS NOT NULL(?0.int2)",
         "true");
   }
@@ -2674,10 +2718,10 @@ class RexProgramTest extends RexProgramTestBase {
   @Test void testSimplifyOrIsNull() {
     // x = 10 OR x IS NULL
     checkSimplify(or(eq(vInt(0), literal(10)), isNull(vInt(0))),
-        "SEARCH(?0.int0, Sarg[10, null])");
+        "SEARCH(?0.int0, Sarg[10 OR NULL])");
     // 10 = x OR x IS NULL
     checkSimplify(or(eq(literal(10), vInt(0)), isNull(vInt(0))),
-        "SEARCH(?0.int0, Sarg[10, null])");
+        "SEARCH(?0.int0, Sarg[10 OR NULL])");
   }
 
   @Test void testSimplifyOrNot() {
@@ -2689,7 +2733,8 @@ 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, IS NULL(?0.int2)))",
+        "OR(null, IS NOT NULL(?0.int1),"
+            + " AND(null, SEARCH(?0.int2, Sarg[NULL])))",
         "IS NOT NULL(?0.int1)",
         "true");
 
@@ -2697,60 +2742,69 @@ 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, IS NULL(?0.int1), OR(null, IS NOT NULL(?0.int2)))",
+        "AND(null, SEARCH(?0.int1, Sarg[NULL]),"
+            + " OR(null, IS NOT NULL(?0.int2)))",
         "false",
-        "IS NULL(?0.int1)");
+        "SEARCH(?0.int1, Sarg[NULL])");
+  }
+
+  private void checkSarg(String message, Sarg sarg,
+      Matcher<Integer> complexityMatcher, Matcher<String> stringMatcher) {
+    assertThat(message, sarg.complexity(), complexityMatcher);
+    assertThat(message, sarg.toString(), stringMatcher);
   }
 
   /** Tests {@link Sarg#complexity()}. */
   @SuppressWarnings("UnstableApiUsage")
   @Test void testSargComplexity() {
-    assertThat("complexity of 'x is not null'",
-        Sarg.of(false, RangeSets.<Integer>rangeSetAll()).complexity(),
-        is(1));
-    assertThat("complexity of 'x is null'",
-        Sarg.of(true, ImmutableRangeSet.<Integer>of()).complexity(),
-        is(1));
-
-    assertThat("complexity of 'x = 1'",
-        Sarg.of(false, ImmutableRangeSet.of(Range.singleton(1))).complexity(),
-        is(1));
-    assertThat("complexity of 'x > 1'",
-        Sarg.of(false, ImmutableRangeSet.of(Range.greaterThan(1)))
-            .complexity(),
-        is(1));
-    assertThat("complexity of 'x >= 1'",
-        Sarg.of(false, ImmutableRangeSet.of(Range.atLeast(1))).complexity(),
-        is(1));
-    assertThat("complexity of 'x > 1 or x is null'",
-        Sarg.of(true, ImmutableRangeSet.of(Range.greaterThan(1))).complexity(),
-        is(2));
-    assertThat("complexity of 'x <> 1'",
-        Sarg.of(false, ImmutableRangeSet.of(Range.singleton(1)).complement())
-            .complexity(),
-        is(1));
-    assertThat("complexity of 'x <> 1 or x is null'",
-        Sarg.of(true, ImmutableRangeSet.of(Range.singleton(1)).complement())
-            .complexity(),
-        is(2));
-    assertThat("complexity of 'x < 10 or x >= 20'",
+    checkSarg("complexity of 'x is not null'",
+        Sarg.of(false, RangeSets.<Integer>rangeSetAll()),
+        is(1), is("Sarg[NOT NULL]"));
+    checkSarg("complexity of 'x is null'",
+        Sarg.of(true, ImmutableRangeSet.<Integer>of()),
+        is(1), is("Sarg[NULL]"));
+    checkSarg("complexity of 'false'",
+        Sarg.of(false, ImmutableRangeSet.<Integer>of()),
+        is(0), is("Sarg[FALSE]"));
+    checkSarg("complexity of 'true'",
+        Sarg.of(true, RangeSets.<Integer>rangeSetAll()),
+        is(2), is("Sarg[TRUE]"));
+
+    checkSarg("complexity of 'x = 1'",
+        Sarg.of(false, ImmutableRangeSet.of(Range.singleton(1))),
+        is(1), is("Sarg[1]"));
+    checkSarg("complexity of 'x > 1'",
+        Sarg.of(false, ImmutableRangeSet.of(Range.greaterThan(1))),
+        is(1), is("Sarg[(1..+\u221E)]"));
+    checkSarg("complexity of 'x >= 1'",
+        Sarg.of(false, ImmutableRangeSet.of(Range.atLeast(1))),
+        is(1), is("Sarg[[1..+\u221E)]"));
+    checkSarg("complexity of 'x > 1 or x is null'",
+        Sarg.of(true, ImmutableRangeSet.of(Range.greaterThan(1))),
+        is(2), is("Sarg[(1..+\u221E) OR NULL]"));
+    checkSarg("complexity of 'x <> 1'",
+        Sarg.of(false, ImmutableRangeSet.of(Range.singleton(1)).complement()),
+        is(1), is("Sarg[(-\u221E..1), (1..+\u221E)]"));
+    checkSarg("complexity of 'x <> 1 or x is null'",
+        Sarg.of(true, ImmutableRangeSet.of(Range.singleton(1)).complement()),
+        is(2), is("Sarg[(-\u221E..1), (1..+\u221E) OR NULL]"));
+    checkSarg("complexity of 'x < 10 or x >= 20'",
         Sarg.of(false,
             ImmutableRangeSet.copyOf(
-                ImmutableList.of(Range.lessThan(10), Range.atLeast(20))))
-            .complexity(),
-        is(2));
-    assertThat("complexity of 'x in (2, 4, 6) or x > 20'",
+                ImmutableList.of(Range.lessThan(10), Range.atLeast(20)))),
+        is(2), is("Sarg[(-\u221E..10), [20..+\u221E)]"));
+    checkSarg("complexity of 'x in (2, 4, 6) or x > 20'",
         Sarg.of(false,
             ImmutableRangeSet.copyOf(
                 Arrays.asList(Range.singleton(2), Range.singleton(4),
-                    Range.singleton(6), Range.greaterThan(20)))).complexity(),
-        is(4));
-    assertThat("complexity of 'x between 3 and 8 or x between 10 and 20'",
+                    Range.singleton(6), Range.greaterThan(20)))),
+        is(4), is("Sarg[2, 4, 6, (20..+\u221E)]"));
+    checkSarg("complexity of 'x between 3 and 8 or x between 10 and 20'",
         Sarg.of(false,
             ImmutableRangeSet.copyOf(
                 Arrays.asList(Range.closed(3, 8),
-                    Range.closed(10, 20)))).complexity(),
-        is(2));
+                    Range.closed(10, 20)))),
+        is(2), is("Sarg[[3..8], [10..20]]"));
   }
 
   @Test void testInterpreter() {
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 024f1f1..be5d678 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -681,7 +681,7 @@ public class RelBuilderTest {
                         builder.equals(builder.field("DEPTNO"),
                             builder.literal(10)),
                         builder.and(builder.isNull(builder.field(6)),
-                            builder.not(builder.isNotNull(builder.field(7))))),
+                            builder.not(builder.isNotNull(builder.field(5))))),
                     builder.equals(builder.field("DEPTNO"),
                         builder.literal(20)),
                     builder.equals(builder.field("DEPTNO"),
@@ -695,7 +695,7 @@ public class RelBuilderTest {
     final String expected = ""
         + "LogicalProject(DEPTNO=[$7], COMM=[CAST($6):SMALLINT NOT NULL], "
         + "$f2=[OR(SEARCH($7, Sarg[20, 30]), AND(null:NULL, =($7, 10), "
-        + "IS NULL($6), IS NULL($7)))], n2=[IS NULL($2)], "
+        + "IS NULL($6), IS NULL($5)))], n2=[IS NULL($2)], "
         + "nn2=[IS NOT NULL($3)], $f5=[20], COMM0=[$6], C=[$6])\n"
         + "  LogicalTableScan(table=[[scott, EMP]])\n";
     assertThat(root, hasTree(expected));
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 b68f023..3311dd9 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -9920,9 +9920,9 @@ LogicalAggregate(group=[{7}], EXPR$1=[COUNT(DISTINCT $10, $12)])
     <TestCase name="testAggregateJoinRemove11">
         <Resource name="sql">
             <![CDATA[select e.deptno, count(distinct e.slacker)
-        from sales.emp e
-        left outer join sales.dept d on e.deptno = d.deptno
-        group by e.deptno]]>
+from sales.emp e
+left outer join sales.dept d on e.deptno = d.deptno
+group by e.deptno]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
@@ -11180,7 +11180,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(=($7, 7), =($0, 10), IS NULL($3))])
+  LogicalFilter(condition=[AND(SEARCH($7, Sarg[7]), SEARCH($0, Sarg[10]), SEARCH($3, Sarg[NULL]))])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -11714,7 +11714,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, <>($9, 0), IS NULL($12))), 10, AND(OR(IS NULL($12), =($9, 0)), OR(>=($10, $9), =($9, 0), IS NOT NULL($12))), 20, 30))])
+    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))])
       LogicalJoin(condition=[=($7, $11)], joinType=[left])
         LogicalJoin(condition=[true], joinType=[inner])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
diff --git a/core/src/test/resources/sql/conditions.iq b/core/src/test/resources/sql/conditions.iq
index 8fb40f9..e35b5f3 100644
--- a/core/src/test/resources/sql/conditions.iq
+++ b/core/src/test/resources/sql/conditions.iq
@@ -299,5 +299,32 @@ select s, (s or s is null), (s and s is not null) from ax;
 
 !ok
 
+!use scott
+
+# Test case for [CALCITE-4352] "IS NOT NULL" condition gets dropped
+select *
+from "scott".emp
+where deptno > 5 AND deptno < 20 AND mgr IS NOT NULL;
++-------+--------+---------+------+------------+---------+------+--------+
+| EMPNO | ENAME  | JOB     | MGR  | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+--------+---------+------+------------+---------+------+--------+
+|  7782 | CLARK  | MANAGER | 7839 | 1981-06-09 | 2450.00 |      |     10 |
+|  7934 | MILLER | CLERK   | 7782 | 1982-01-23 | 1300.00 |      |     10 |
++-------+--------+---------+------+------------+---------+------+--------+
+(2 rows)
+
+!ok
+
+select *
+from "scott".emp
+where deptno > 5 AND deptno < 20 AND mgr IS NULL;
++-------+-------+-----------+-----+------------+---------+------+--------+
+| EMPNO | ENAME | JOB       | MGR | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+-------+-----------+-----+------------+---------+------+--------+
+|  7839 | KING  | PRESIDENT |     | 1981-11-17 | 5000.00 |      |     10 |
++-------+-------+-----------+-----+------------+---------+------+--------+
+(1 row)
+
+!ok
 
 # End conditions.iq
diff --git a/core/src/test/resources/sql/sub-query.iq b/core/src/test/resources/sql/sub-query.iq
index a7fc8a3..e5cd646 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=[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])
+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])
   EnumerableHashJoin(condition=[=($0, $3)], joinType=[left])
     EnumerableNestedLoopJoin(condition=[true], joinType=[inner])
       EnumerableUnion(all=[true])