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 2021/03/02 20:29:10 UTC

[calcite] branch master updated (3cfeba8 -> 00d1086)

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

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


    from 3cfeba8  [CALCITE-2000] UNNEST a collection that has a field with nested data generates an Exception
     add 0c64a58  Fuzz testing for SEARCH operator, and refactor RexSimplify
     new 00d1086  [CALCITE-4446] Implement three-valued logic for SEARCH operator

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../apache/calcite/rel/rel2sql/SqlImplementor.java |   3 +-
 .../java/org/apache/calcite/rex/RexAnalyzer.java   |   7 +-
 .../java/org/apache/calcite/rex/RexBuilder.java    |   8 +-
 .../main/java/org/apache/calcite/rex/RexCall.java  |   6 +-
 .../org/apache/calcite/rex/RexInterpreter.java     |  64 +++++++
 .../java/org/apache/calcite/rex/RexSimplify.java   | 211 +++++++++++----------
 .../java/org/apache/calcite/rex/RexUnknownAs.java  |  17 ++
 .../main/java/org/apache/calcite/rex/RexUtil.java  |  63 ++++--
 .../main/java/org/apache/calcite/sql/SqlKind.java  |  11 --
 .../apache/calcite/sql/fun/SqlSearchOperator.java  |  10 +-
 .../main/java/org/apache/calcite/util/Sarg.java    | 181 +++++++++++++++---
 .../calcite/rel/rel2sql/RelToSqlConverterTest.java |  11 ++
 .../apache/calcite/rex/RexProgramBuilderBase.java  |   4 +
 .../org/apache/calcite/rex/RexProgramTest.java     | 143 ++++++++------
 .../org/apache/calcite/rex/RexProgramTestBase.java |  36 ++--
 .../org/apache/calcite/test/RelBuilderTest.java    |   6 +-
 .../org/apache/calcite/test/fuzzer/RexFuzzer.java  |  98 +++++++++-
 .../calcite/test/fuzzer/RexProgramFuzzyTest.java   |   1 -
 .../org/apache/calcite/test/RelOptRulesTest.xml    |   2 +-
 .../calcite/test/DruidDateRangeRulesTest.java      |   4 +-
 20 files changed, 643 insertions(+), 243 deletions(-)


[calcite] 01/01: [CALCITE-4446] Implement three-valued logic for SEARCH operator

Posted by jh...@apache.org.
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 00d1086c7003115a9ed9dcc874051f515b7b5aae
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Wed Dec 23 13:32:41 2020 -0800

    [CALCITE-4446] Implement three-valued logic for SEARCH operator
    
    Close apache/calcite#2357
---
 .../apache/calcite/rel/rel2sql/SqlImplementor.java |   3 +-
 .../java/org/apache/calcite/rex/RexAnalyzer.java   |   7 +-
 .../java/org/apache/calcite/rex/RexBuilder.java    |   8 +-
 .../main/java/org/apache/calcite/rex/RexCall.java  |   6 +-
 .../org/apache/calcite/rex/RexInterpreter.java     |  64 ++++++++
 .../java/org/apache/calcite/rex/RexSimplify.java   | 160 +++++++++---------
 .../java/org/apache/calcite/rex/RexUnknownAs.java  |  17 ++
 .../main/java/org/apache/calcite/rex/RexUtil.java  |  63 +++++--
 .../apache/calcite/sql/fun/SqlSearchOperator.java  |  10 +-
 .../main/java/org/apache/calcite/util/Sarg.java    | 181 ++++++++++++++++++---
 .../apache/calcite/rex/RexProgramBuilderBase.java  |   4 +
 .../org/apache/calcite/rex/RexProgramTest.java     | 132 ++++++++-------
 .../org/apache/calcite/rex/RexProgramTestBase.java |  36 ++--
 .../org/apache/calcite/test/RelBuilderTest.java    |   6 +-
 .../org/apache/calcite/test/fuzzer/RexFuzzer.java  |  17 +-
 .../org/apache/calcite/test/RelOptRulesTest.xml    |   2 +-
 .../calcite/test/DruidDateRangeRulesTest.java      |   4 +-
 17 files changed, 499 insertions(+), 221 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
index 5ae29ef..4632853 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
@@ -47,6 +47,7 @@ import org.apache.calcite.rex.RexPatternFieldRef;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexSubQuery;
+import org.apache.calcite.rex.RexUnknownAs;
 import org.apache.calcite.rex.RexWindow;
 import org.apache.calcite.rex.RexWindowBound;
 import org.apache.calcite.sql.JoinType;
@@ -892,7 +893,7 @@ public abstract class SqlImplementor {
         RexNode operand, RelDataType type, Sarg<C> sarg) {
       final List<SqlNode> orList = new ArrayList<>();
       final SqlNode operandSql = toSql(program, operand);
-      if (sarg.containsNull) {
+      if (sarg.nullAs == RexUnknownAs.TRUE) {
         orList.add(SqlStdOperatorTable.IS_NULL.createCall(POS, operandSql));
       }
       if (sarg.isPoints()) {
diff --git a/core/src/main/java/org/apache/calcite/rex/RexAnalyzer.java b/core/src/main/java/org/apache/calcite/rex/RexAnalyzer.java
index 65afaec..9d46f83 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexAnalyzer.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexAnalyzer.java
@@ -131,11 +131,14 @@ public class RexAnalyzer {
     }
 
     @Override public Void visitCall(RexCall call) {
-      if (!RexInterpreter.SUPPORTED_SQL_KIND.contains(call.getKind())) {
+      switch (call.getKind()) {
+      case CAST:
+      case OTHER_FUNCTION:
         ++unsupportedCount;
         return null;
+      default:
+        return super.visitCall(call);
       }
-      return super.visitCall(call);
     }
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
index 7bd3536..6a6e599 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
@@ -1330,7 +1330,7 @@ public class RexBuilder {
    * otherwise creates a disjunction, "arg = point0 OR arg = point1 OR ...". */
   public RexNode makeIn(RexNode arg, List<? extends RexNode> ranges) {
     if (areAssignable(arg, ranges)) {
-      final Sarg sarg = toSarg(Comparable.class, ranges, false);
+      final Sarg sarg = toSarg(Comparable.class, ranges, RexUnknownAs.UNKNOWN);
       if (sarg != null) {
         final RexNode range0 = ranges.get(0);
         return makeCall(SqlStdOperatorTable.SEARCH,
@@ -1369,7 +1369,7 @@ public class RexBuilder {
         && upperValue != null
         && areAssignable(arg, Arrays.asList(lower, upper))) {
       final Sarg sarg =
-          Sarg.of(false,
+          Sarg.of(RexUnknownAs.UNKNOWN,
               ImmutableRangeSet.<Comparable>of(
                   Range.closed(lowerValue, upperValue)));
       return makeCall(SqlStdOperatorTable.SEARCH, arg,
@@ -1384,7 +1384,7 @@ public class RexBuilder {
    * not possible. */
   @SuppressWarnings({"BetaApi", "UnstableApiUsage"})
   private static <C extends Comparable<C>> @Nullable Sarg<C> toSarg(Class<C> clazz,
-      List<? extends RexNode> ranges, boolean containsNull) {
+      List<? extends RexNode> ranges, RexUnknownAs unknownAs) {
     if (ranges.isEmpty()) {
       // Cannot convert an empty list to a Sarg (by this interface, at least)
       // because we use the type of the first element.
@@ -1398,7 +1398,7 @@ public class RexBuilder {
       }
       rangeSet.add(Range.singleton(value));
     }
-    return Sarg.of(containsNull, rangeSet);
+    return Sarg.of(unknownAs, rangeSet);
   }
 
   private static <C extends Comparable<C>> @Nullable C toComparable(Class<C> clazz,
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 5347a10..2526d7d 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCall.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCall.java
@@ -214,7 +214,8 @@ public class RexCall extends RexNode {
     case SEARCH:
       final Sarg sarg = ((RexLiteral) operands.get(1)).getValueAs(Sarg.class);
       return requireNonNull(sarg, "sarg").isAll()
-          && (sarg.containsNull || !operands.get(0).getType().isNullable());
+          && (sarg.nullAs == RexUnknownAs.TRUE
+              || !operands.get(0).getType().isNullable());
     default:
       return false;
     }
@@ -235,7 +236,8 @@ public class RexCall extends RexNode {
     case SEARCH:
       final Sarg sarg = ((RexLiteral) operands.get(1)).getValueAs(Sarg.class);
       return requireNonNull(sarg, "sarg").isNone()
-          && (!sarg.containsNull || !operands.get(0).getType().isNullable());
+          && (sarg.nullAs == RexUnknownAs.FALSE
+              || !operands.get(0).getType().isNullable());
     default:
       return false;
     }
diff --git a/core/src/main/java/org/apache/calcite/rex/RexInterpreter.java b/core/src/main/java/org/apache/calcite/rex/RexInterpreter.java
index ac234e2..8a4012e 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexInterpreter.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexInterpreter.java
@@ -20,11 +20,19 @@ import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.rel.metadata.NullSentinel;
+import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.RangeSets;
+import org.apache.calcite.util.Sarg;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.RangeSet;
 
 import org.checkerframework.checker.nullness.qual.Nullable;
 
@@ -209,6 +217,10 @@ public class RexInterpreter implements RexVisitor<Comparable> {
       return ceil(call, values);
     case EXTRACT:
       return extract(values);
+    case LIKE:
+      return like(values);
+    case SEARCH:
+      return search(call.operands.get(1).getType().getSqlTypeName(), values);
     default:
       throw unbound(call);
     }
@@ -231,6 +243,58 @@ public class RexInterpreter implements RexVisitor<Comparable> {
     return DateTimeUtils.unixDateExtract(timeUnitRange, v2);
   }
 
+  private static Comparable like(List<Comparable> values) {
+    if (containsNull(values)) {
+      return N;
+    }
+    final NlsString value = (NlsString) values.get(0);
+    final NlsString pattern = (NlsString) values.get(1);
+    switch (values.size()) {
+    case 2:
+      return SqlFunctions.like(value.getValue(), pattern.getValue());
+    case 3:
+      final NlsString escape = (NlsString) values.get(2);
+      return SqlFunctions.like(value.getValue(), pattern.getValue(),
+          escape.getValue());
+    default:
+      throw new AssertionError();
+    }
+  }
+
+  @SuppressWarnings({"BetaApi", "rawtypes", "unchecked", "UnstableApiUsage"})
+  private static Comparable search(SqlTypeName typeName, List<Comparable> values) {
+    final Comparable value = values.get(0);
+    final Sarg sarg = (Sarg) values.get(1);
+    if (value == N) {
+      switch (sarg.nullAs) {
+      case FALSE:
+        return false;
+      case TRUE:
+        return true;
+      default:
+        return N;
+      }
+    }
+    return translate(sarg.rangeSet, typeName).contains(value);
+  }
+
+  /** Translates the values in a RangeSet from literal format to runtime format.
+   * For example the DATE SQL type uses DateString for literals and Integer at
+   * runtime. */
+  @SuppressWarnings({"BetaApi", "rawtypes", "unchecked", "UnstableApiUsage"})
+  private static RangeSet translate(RangeSet rangeSet, SqlTypeName typeName) {
+    switch (typeName) {
+    case DATE:
+      return RangeSets.copy(rangeSet, DateString::getDaysSinceEpoch);
+    case TIME:
+      return RangeSets.copy(rangeSet, TimeString::getMillisOfDay);
+    case TIMESTAMP:
+      return RangeSets.copy(rangeSet, TimestampString::getMillisSinceEpoch);
+    default:
+      return rangeSet;
+    }
+  }
+
   private static Comparable coalesce(List<Comparable> values) {
     for (Comparable value : values) {
       if (value != N) {
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 a070734..0579025 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -336,7 +336,12 @@ public class RexSimplify {
     if (e.operands.get(1) instanceof RexLiteral) {
       final RexLiteral literal = (RexLiteral) e.operands.get(1);
       if ("%".equals(literal.getValueAs(String.class))) {
-        return rexBuilder.makeLiteral(true);
+        RexNode x = e.operands.get(0);
+        // "x LIKE '%'" simplifies to "UNKNOWN OR x IS NOT NULL"
+        return simplify(
+            rexBuilder.makeCall(SqlStdOperatorTable.OR,
+                rexBuilder.makeNullLiteral(e.getType()),
+                rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, x)));
       }
     }
     return simplifyGenericNode(e);
@@ -1254,7 +1259,7 @@ public class RexSimplify {
 
     // prepare all condition/branches for boolean interpretation
     // It's done here make these interpretation changes available to case2or simplifications
-    // but not interfere with the normal simplifcation recursion
+    // but not interfere with the normal simplification recursion
     List<CaseBranch> branches = new ArrayList<>();
     for (CaseBranch branch : inputBranches) {
       if ((branches.size() > 0 && !isSafeExpression(branch.cond))
@@ -1342,9 +1347,10 @@ public class RexSimplify {
 
     final SargCollector sargCollector = new SargCollector(rexBuilder, true);
     operands.forEach(t -> sargCollector.accept(t, terms));
-    if (sargCollector.needToFix(unknownAs)) {
+    if (sargCollector.needToFix()) {
       operands.clear();
-      terms.forEach(t -> operands.add(sargCollector.fix(rexBuilder, t)));
+      terms.forEach(t ->
+          operands.add(SargCollector.fix(rexBuilder, t, unknownAs)));
     }
     terms.clear();
 
@@ -1816,9 +1822,10 @@ 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.needToFix(unknownAs)) {
+    if (sargCollector.needToFix()) {
       terms.clear();
-      newTerms.forEach(t -> terms.add(sargCollector.fix(rexBuilder, t)));
+      newTerms.forEach(t ->
+          terms.add(SargCollector.fix(rexBuilder, t, unknownAs)));
     }
 
     // CALCITE-3198 Auxiliary map to simplify cases like:
@@ -1952,9 +1959,10 @@ public class RexSimplify {
         }
       }
       if (!v0.equals(v1)) {
-        throw new AssertionError("result mismatch: when applied to " + map
-            + ", " + before + " yielded " + v0
-            + ", and " + simplified + " yielded " + v1);
+        throw new AssertionError("result mismatch (unknown as "
+            + unknownAs + "): when applied to " + map + ",\n"
+            + before + " yielded " + v0 + ";\n"
+            + simplified + " yielded " + v1);
       }
     }
   }
@@ -1965,22 +1973,19 @@ public class RexSimplify {
     if (call.getOperands().get(1) instanceof RexLiteral) {
       RexLiteral literal = (RexLiteral) call.getOperands().get(1);
       final Sarg sarg = castNonNull(literal.getValueAs(Sarg.class));
-      if (sarg.isAll()
-          && (sarg.containsNull || !a.getType().isNullable())) {
-        // SEARCH(x, [TRUE])  -> TRUE
-        // SEARCH(x, [NOT NULL])  -> TRUE if x's type is NOT NULL
-        return rexBuilder.makeLiteral(true);
+      if (sarg.isAll() || sarg.isNone()) {
+        return RexUtil.simpleSarg(rexBuilder, a, sarg, unknownAs);
       }
       // Remove null from sarg if the left-hand side is never null
-      if (sarg.containsNull) {
+      if (sarg.nullAs != UNKNOWN) {
         final RexNode simplified = simplifyIs1(SqlKind.IS_NULL, a, unknownAs);
         if (simplified != null
             && simplified.isAlwaysFalse()) {
-          final Sarg sarg2 = Sarg.of(false, sarg.rangeSet);
+          final Sarg sarg2 = Sarg.of(UNKNOWN, sarg.rangeSet);
           final RexLiteral literal2 =
               rexBuilder.makeLiteral(sarg2, literal.getType(),
                   literal.getTypeName());
-          // Now we've strengthened containsNull to false, try to simplify again
+          // Now we've strengthened the Sarg, try to simplify again
           return simplifySearch(
               call.clone(call.type, ImmutableList.of(a, literal2)),
               unknownAs);
@@ -2697,22 +2702,13 @@ public class RexSimplify {
       final RexSargBuilder b =
           map.computeIfAbsent(e, e2 ->
               addFluent(newTerms, new RexSargBuilder(e2, rexBuilder, negate)));
-      switch (kind) {
+      switch (negate ? kind.negate() : kind) {
       case IS_NULL:
-        if (negate) {
-          ++b.nullTermCount;
-          b.addAll();
-        } else {
-          ++b.nullTermCount;
-        }
+        b.nullAs = b.nullAs.or(TRUE);
         return true;
       case IS_NOT_NULL:
-        if (negate) {
-          ++b.notNullTermCount;
-        } else {
-          ++b.notNullTermCount;
-          b.addAll();
-        }
+        b.nullAs = b.nullAs.or(FALSE);
+        b.addAll();
         return true;
       default:
         throw new AssertionError("unexpected " + kind);
@@ -2764,35 +2760,8 @@ 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) {
+    boolean needToFix() {
       // Fix and converts to SEARCH if:
       // 1. A Sarg has complexity greater than 1;
       // 2. The terms are reduced as simpler Sarg points;
@@ -2802,15 +2771,9 @@ public class RexSimplify {
       // 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()));
+      return builders.stream().anyMatch(b -> b.build(false).complexity() > 1)
+          || newTermsCount == 1
+          && builders.stream().allMatch(b -> simpleSarg(b.build()));
     }
 
     /**
@@ -2823,14 +2786,16 @@ public class RexSimplify {
 
     /** If a term is a call to {@code SEARCH} on a {@link RexSargBuilder},
      * converts it to a {@code SEARCH} on a {@link Sarg}. */
-    static RexNode fix(RexBuilder rexBuilder, RexNode term) {
+    static RexNode fix(RexBuilder rexBuilder, RexNode term,
+        RexUnknownAs unknownAs) {
       if (term instanceof RexSargBuilder) {
         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 RexUtil.sargRef(rexBuilder, sargBuilder.ref, sarg,
+              term.getType(), unknownAs);
         }
         return rexBuilder.makeCall(SqlStdOperatorTable.SEARCH, sargBuilder.ref,
             rexBuilder.makeSearchArgumentLiteral(sarg, term.getType()));
@@ -2844,7 +2809,22 @@ public class RexSimplify {
    * traverses a list of OR or AND terms.
    *
    * <p>The {@link SargCollector#fix} method converts it to an immutable
-   * literal. */
+   * literal.
+   *
+   * <p>The {@link #nullAs} field will become {@link Sarg#nullAs}, as follows:
+   *
+   * <ul>
+   * <li>If there is at least one term that returns TRUE when the argument
+   * is NULL, then the overall value will be TRUE; failing that,
+   * <li>if there is at least one term that returns UNKNOWN when the argument
+   * is NULL, then the overall value will be UNKNOWN; failing that,
+   * <li>the value will be FALSE.
+   * </ul>
+   *
+   * <p>This is analogous to the behavior of OR in three-valued logic:
+   * {@code TRUE OR UNKNOWN OR FALSE} returns {@code TRUE};
+   * {@code UNKNOWN OR FALSE OR UNKNOWN} returns {@code UNKNOWN};
+   * {@code FALSE OR FALSE} returns {@code FALSE}. */
   @SuppressWarnings("BetaApi")
   private static class RexSargBuilder extends RexNode {
     final RexNode ref;
@@ -2852,8 +2832,7 @@ public class RexSimplify {
     final boolean negate;
     final List<RelDataType> types = new ArrayList<>();
     final RangeSet<Comparable> rangeSet = TreeRangeSet.create();
-    int notNullTermCount;
-    int nullTermCount;
+    RexUnknownAs nullAs = FALSE;
 
     RexSargBuilder(RexNode ref, RexBuilder rexBuilder, boolean negate) {
       this.ref = requireNonNull(ref, "ref");
@@ -2863,8 +2842,7 @@ public class RexSimplify {
 
     @Override public String toString() {
       return "SEARCH(" + ref + ", " + (negate ? "NOT " : "") + rangeSet
-          + ", " + (nullTermCount + notNullTermCount)
-          + " terms of which " + nullTermCount + " allow null)";
+          + "; NULL AS " + nullAs + ")";
     }
 
     <C extends Comparable<C>> Sarg<C> build() {
@@ -2873,11 +2851,11 @@ public class RexSimplify {
 
     @SuppressWarnings({"rawtypes", "unchecked", "UnstableApiUsage"})
     <C extends Comparable<C>> Sarg<C> build(boolean negate) {
+      final RangeSet<C> r = (RangeSet) this.rangeSet;
       if (negate) {
-        return Sarg.of(notNullTermCount == 0,
-            (RangeSet) rangeSet.complement());
+        return Sarg.of(nullAs.negate(), r.complement());
       } else {
-        return Sarg.of(nullTermCount > 0, (RangeSet) rangeSet);
+        return Sarg.of(nullAs, r);
       }
     }
 
@@ -2915,16 +2893,32 @@ public class RexSimplify {
     void addRange(Range<Comparable> range, RelDataType type) {
       types.add(type);
       rangeSet.add(range);
-      ++notNullTermCount;
+      nullAs = nullAs.or(UNKNOWN);
     }
 
+    @SuppressWarnings({"UnstableApiUsage", "rawtypes", "unchecked"})
     void addSarg(Sarg sarg, boolean negate, RelDataType type) {
-      types.add(type);
-      rangeSet.addAll(negate ? sarg.rangeSet.complement() : sarg.rangeSet);
-      if (sarg.containsNull) {
-        ++nullTermCount;
+      final RangeSet r;
+      final RexUnknownAs nullAs;
+      if (negate) {
+        r = sarg.rangeSet.complement();
+        nullAs = sarg.nullAs.negate();
       } else {
-        ++notNullTermCount;
+        r = sarg.rangeSet;
+        nullAs = sarg.nullAs;
+      }
+      types.add(type);
+      rangeSet.addAll(r);
+      switch (nullAs) {
+      case TRUE:
+        this.nullAs = this.nullAs.or(TRUE);
+        break;
+      case FALSE:
+        this.nullAs = this.nullAs.or(FALSE);
+        break;
+      case UNKNOWN:
+        this.nullAs = this.nullAs.or(UNKNOWN);
+        break;
       }
     }
   }
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUnknownAs.java b/core/src/main/java/org/apache/calcite/rex/RexUnknownAs.java
index 99841c3..6ec514e 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUnknownAs.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUnknownAs.java
@@ -103,4 +103,21 @@ public enum RexUnknownAs {
       return UNKNOWN;
     }
   }
+
+  /** Combines this with another {@code RexUnknownAs} in the same way as the
+   * three-valued logic of OR.
+   *
+   * <p>For example, {@code TRUE or FALSE} returns {@code TRUE};
+   * {@code FALSE or UNKNOWN} returns {@code UNKNOWN}. */
+  public RexUnknownAs or(RexUnknownAs other) {
+    switch (this) {
+    case TRUE:
+      return this;
+    case UNKNOWN:
+      return other == TRUE ? other : this;
+    case FALSE:
+    default:
+      return other;
+    }
+  }
 }
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 d02da01..54899dc 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -597,17 +597,14 @@ public class RexUtil {
   }
 
   @SuppressWarnings("BetaApi")
-  public 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);
-      }
+  public static <C extends Comparable<C>> RexNode sargRef(RexBuilder rexBuilder,
+      RexNode ref, Sarg<C> sarg, RelDataType type, RexUnknownAs unknownAs) {
+    if (sarg.isAll() || sarg.isNone()) {
+      return simpleSarg(rexBuilder, ref, sarg, unknownAs);
     }
     final List<RexNode> orList = new ArrayList<>();
-    if (sarg.containsNull) {
+    if (sarg.nullAs == RexUnknownAs.TRUE
+        && unknownAs == RexUnknownAs.UNKNOWN) {
       orList.add(rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, ref));
     }
     if (sarg.isPoints()) {
@@ -631,7 +628,50 @@ public class RexUtil {
           new RangeToRex<>(ref, orList, rexBuilder, type);
       RangeSets.forEach(sarg.rangeSet, consumer);
     }
-    return composeDisjunction(rexBuilder, orList);
+    RexNode node = composeDisjunction(rexBuilder, orList);
+    if (sarg.nullAs == RexUnknownAs.FALSE
+        && unknownAs == RexUnknownAs.UNKNOWN) {
+      node =
+          rexBuilder.makeCall(SqlStdOperatorTable.AND,
+              rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, ref),
+              node);
+    }
+    return node;
+  }
+
+  /** Expands an 'all' or 'none' sarg. */
+  public static <C extends Comparable<C>> RexNode simpleSarg(RexBuilder rexBuilder,
+      RexNode ref, Sarg<C> sarg, RexUnknownAs unknownAs) {
+    assert sarg.isAll() || sarg.isNone();
+    final RexUnknownAs nullAs =
+        sarg.nullAs == RexUnknownAs.UNKNOWN ? unknownAs
+            : sarg.nullAs;
+    if (sarg.isAll()) {
+      switch (nullAs) {
+      case TRUE:
+        return rexBuilder.makeLiteral(true);
+      case FALSE:
+        return rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, ref);
+      case UNKNOWN:
+        // "x IS NOT NULL OR UNKNOWN"
+        return rexBuilder.makeCall(SqlStdOperatorTable.OR,
+            rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, ref),
+            rexBuilder.makeNullLiteral(
+                rexBuilder.typeFactory.createSqlType(SqlTypeName.BOOLEAN)));
+      }
+    }
+    if (sarg.isNone()) {
+      switch (nullAs) {
+      case TRUE:
+        return rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, ref);
+      case FALSE:
+        return rexBuilder.makeLiteral(false);
+      case UNKNOWN:
+        // "CASE WHEN x IS NULL THEN UNKNOWN ELSE FALSE END", or "x <> x"
+        return rexBuilder.makeCall(SqlStdOperatorTable.NOT_EQUALS, ref, ref);
+      }
+    }
+    throw new AssertionError();
   }
 
   private static RexNode deref(@Nullable RexProgram program, RexNode node) {
@@ -3054,7 +3094,8 @@ public class RexUtil {
             (RexLiteral) deref(program, call.operands.get(1));
         final Sarg sarg = requireNonNull(literal.getValueAs(Sarg.class), "Sarg");
         if (maxComplexity < 0 || sarg.complexity() < maxComplexity) {
-          return sargRef(rexBuilder, ref, sarg, literal.getType());
+          return sargRef(rexBuilder, ref, sarg, literal.getType(),
+              RexUnknownAs.UNKNOWN);
         }
         // Sarg is complex (therefore useful); fall through
       default:
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 dc20e72..8fcd9ed 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
@@ -17,6 +17,7 @@
 package org.apache.calcite.sql.fun;
 
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexUnknownAs;
 import org.apache.calcite.sql.SqlInternalOperator;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperatorBinding;
@@ -44,16 +45,17 @@ class SqlSearchOperator extends SqlInternalOperator {
    * 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
+   * <p>If {@link Sarg#nullAs} is TRUE or FALSE, SEARCH will never return
+   * UNKNOWN. For example, {@code SEARCH(x, Sarg[10; NULL AS UNKNOWN])} 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()
-        && !getOperandLiteralValueOrThrow(binding, 1, Sarg.class).containsNull;
+        && getOperandLiteralValueOrThrow(binding, 1, Sarg.class).nullAs
+        == RexUnknownAs.UNKNOWN;
     return binding.getTypeFactory().createTypeWithNullability(type, nullable);
   }
 }
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 55ed4d0..ef3822c 100644
--- a/core/src/main/java/org/apache/calcite/util/Sarg.java
+++ b/core/src/main/java/org/apache/calcite/util/Sarg.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.util;
 
 import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rex.RexUnknownAs;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 
 import com.google.common.collect.ImmutableRangeSet;
@@ -63,36 +64,111 @@ import java.util.function.BiConsumer;
  *
  * @see SqlStdOperatorTable#SEARCH
  */
-@SuppressWarnings({"BetaApi", "type.argument.type.incompatible"})
+@SuppressWarnings({"BetaApi", "type.argument.type.incompatible", "UnstableApiUsage"})
 public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
   public final RangeSet<C> rangeSet;
+  public final RexUnknownAs nullAs;
+  @Deprecated // to be removed before 1.28
   public final boolean containsNull;
   public final int pointCount;
 
-  private Sarg(ImmutableRangeSet<C> rangeSet, boolean containsNull) {
+  /** Returns FALSE for all null and not-null values.
+   *
+   * <p>{@code SEARCH(x, FALSE)} is equivalent to {@code FALSE}. */
+  private static final SpecialSarg FALSE =
+      new SpecialSarg(ImmutableRangeSet.of(), RexUnknownAs.FALSE,
+          "Sarg[FALSE]", 2);
+
+  /** Returns TRUE for all not-null values, FALSE for null.
+   *
+   * <p>{@code SEARCH(x, IS_NOT_NULL)} is equivalent to
+   * {@code x IS NOT NULL}. */
+  private static final SpecialSarg IS_NOT_NULL =
+      new SpecialSarg(ImmutableRangeSet.of().complement(), RexUnknownAs.FALSE,
+          "Sarg[IS NOT NULL]", 3);
+
+  /** Returns FALSE for all not-null values, TRUE for null.
+   *
+   * <p>{@code SEARCH(x, IS_NULL)} is equivalent to {@code x IS NULL}. */
+  private static final SpecialSarg IS_NULL =
+      new SpecialSarg(ImmutableRangeSet.of(), RexUnknownAs.TRUE,
+          "Sarg[IS NULL]", 4);
+
+  /** Returns TRUE for all null and not-null values.
+   *
+   * <p>{@code SEARCH(x, TRUE)} is equivalent to {@code TRUE}. */
+  private static final SpecialSarg TRUE =
+      new SpecialSarg(ImmutableRangeSet.of().complement(), RexUnknownAs.TRUE,
+          "Sarg[TRUE]", 5);
+
+  /** Returns FALSE for all not-null values, UNKNOWN for null.
+   *
+   * <p>{@code SEARCH(x, NOT_EQUAL)} is equivalent to {@code x <> x}. */
+  private static final SpecialSarg NOT_EQUAL =
+      new SpecialSarg(ImmutableRangeSet.of(), RexUnknownAs.UNKNOWN,
+          "Sarg[<>]", 6);
+
+  /** Returns TRUE for all not-null values, UNKNOWN for null.
+   *
+   * <p>{@code SEARCH(x, EQUAL)} is equivalent to {@code x = x}. */
+  private static final SpecialSarg EQUAL =
+      new SpecialSarg(ImmutableRangeSet.of().complement(), RexUnknownAs.UNKNOWN,
+          "Sarg[=]", 7);
+
+  private Sarg(ImmutableRangeSet<C> rangeSet, RexUnknownAs nullAs) {
     this.rangeSet = Objects.requireNonNull(rangeSet, "rangeSet");
-    this.containsNull = containsNull;
+    this.nullAs = Objects.requireNonNull(nullAs, "nullAs");
+    this.containsNull = nullAs == RexUnknownAs.TRUE;
     this.pointCount = RangeSets.countPoints(rangeSet);
   }
 
-  /** Creates a search argument. */
+  @Deprecated // to be removed before 2.0
   public static <C extends Comparable<C>> Sarg<C> of(boolean containsNull,
       RangeSet<C> rangeSet) {
-    return new Sarg<>(ImmutableRangeSet.copyOf(rangeSet), containsNull);
+    return of(containsNull ? RexUnknownAs.TRUE : RexUnknownAs.UNKNOWN,
+        rangeSet);
+  }
+
+  /** Creates a search argument. */
+  public static <C extends Comparable<C>> Sarg<C> of(RexUnknownAs nullAs,
+      RangeSet<C> rangeSet) {
+    if (rangeSet.isEmpty()) {
+      switch (nullAs) {
+      case FALSE:
+        return FALSE;
+      case TRUE:
+        return IS_NULL;
+      default:
+        return NOT_EQUAL;
+      }
+    }
+    if (rangeSet.equals(RangeSets.rangeSetAll())) {
+      switch (nullAs) {
+      case FALSE:
+        return IS_NOT_NULL;
+      case TRUE:
+        return TRUE;
+      default:
+        return EQUAL;
+      }
+    }
+    return new Sarg<>(ImmutableRangeSet.copyOf(rangeSet), nullAs);
   }
 
   /**
    * {@inheritDoc}
    *
-   * <p>Produces a similar result to {@link RangeSet}, but adds ", null"
-   * if nulls are matched, and simplifies point ranges. For example,
-   * the Sarg that allows the range set
+   * <p>Produces a similar result to {@link RangeSet},
+   * but adds "; NULL AS FALSE" or "; NULL AS TRUE" to indicate {@link #nullAs},
+   * and simplifies point ranges.
+   *
+   * <p>For example, the Sarg that allows the range set
    *
    * <blockquote>{@code [[7..7], [9..9], (10..+∞)]}</blockquote>
    *
-   * and also null is printed as
+   * <p>and also null is printed as
    *
-   * <blockquote>{@code Sarg[7, 9, (10..+∞) OR NULL]}</blockquote>
+   * <blockquote>{@code Sarg[7, 9, (10..+∞); NULL AS TRUE]}</blockquote>
    */
   @Override public String toString() {
     final StringBuilder sb = new StringBuilder();
@@ -104,12 +180,6 @@ 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) -> {
@@ -118,10 +188,16 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
       }
       RangeSets.forEach(r, printer);
     });
-    if (containsNull) {
-      sb.append(" OR NULL");
+    switch (nullAs) {
+    case FALSE:
+      return sb.append("; NULL AS FALSE]");
+    case TRUE:
+      return sb.append("; NULL AS TRUE]");
+    case UNKNOWN:
+      return sb.append("]");
+    default:
+      throw new AssertionError();
     }
-    return sb.append("]");
   }
 
   @Override public int compareTo(Sarg<C> o) {
@@ -129,26 +205,26 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
   }
 
   @Override public int hashCode() {
-    return RangeSets.hashCode(rangeSet) * 31 + (containsNull ? 2 : 3);
+    return RangeSets.hashCode(rangeSet) * 31 + nullAs.ordinal();
   }
 
   @Override public boolean equals(@Nullable Object o) {
     return o == this
         || o instanceof Sarg
-        && containsNull == ((Sarg) o).containsNull
+        && nullAs == ((Sarg) o).nullAs
         && rangeSet.equals(((Sarg) o).rangeSet);
   }
 
   /** Returns whether this Sarg includes all values (including or not including
    * null). */
   public boolean isAll() {
-    return rangeSet.equals(RangeSets.rangeSetAll());
+    return false;
   }
 
   /** Returns whether this Sarg includes no values (including or not including
    * null). */
   public boolean isNone() {
-    return rangeSet.isEmpty();
+    return false;
   }
 
   /** Returns whether this Sarg is a collection of 1 or more points (and perhaps
@@ -198,7 +274,7 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
     } else {
       complexity = rangeSet.asRanges().size();
     }
-    if (containsNull) {
+    if (nullAs == RexUnknownAs.TRUE) {
       ++complexity;
     }
     return complexity;
@@ -206,6 +282,61 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
 
   /** Returns a Sarg that matches a value if and only this Sarg does not. */
   public Sarg negate() {
-    return Sarg.of(!containsNull, rangeSet.complement());
+    return Sarg.of(nullAs.negate(), rangeSet.complement());
+  }
+
+  /** Sarg whose range is all or none.
+   *
+   * <p>There are only 6 instances: {all, none} * {true, false, unknown}.
+   *
+   * @param <C> Value type */
+  private static class SpecialSarg<C extends Comparable<C>> extends Sarg<C> {
+    final String name;
+    final int ordinal;
+
+    SpecialSarg(ImmutableRangeSet<C> rangeSet, RexUnknownAs nullAs, String name,
+        int ordinal) {
+      super(rangeSet, nullAs);
+      this.name = name;
+      this.ordinal = ordinal;
+      assert rangeSet.isEmpty() == ((ordinal & 1) == 0);
+      assert rangeSet.equals(RangeSets.rangeSetAll()) == ((ordinal & 1) == 1);
+    }
+
+    @Override public boolean equals(@Nullable Object o) {
+      return this == o;
+    }
+
+    @Override public int hashCode() {
+      return ordinal;
+    }
+
+    @Override public boolean isAll() {
+      return (ordinal & 1) == 1;
+    }
+
+    @Override public boolean isNone() {
+      return (ordinal & 1) == 0;
+    }
+
+    @Override public int complexity() {
+      switch (ordinal) {
+      case 2: // Sarg[FALSE]
+        return 0; // for backwards compatibility
+      case 5: // Sarg[TRUE]
+        return 2; // for backwards compatibility
+      default:
+        return 1;
+      }
+    }
+
+    @Override public StringBuilder printTo(StringBuilder sb,
+        BiConsumer<StringBuilder, C> valuePrinter) {
+      return sb.append(name);
+    }
+
+    @Override public String toString() {
+      return name;
+    }
   }
 }
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 f1a48a0..6b25f2c 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramBuilderBase.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramBuilderBase.java
@@ -187,18 +187,22 @@ public abstract class RexProgramBuilderBase {
   }
 
   protected RexNode isFalse(RexNode node) {
+    assert node.getType().getSqlTypeName() == SqlTypeName.BOOLEAN;
     return rexBuilder.makeCall(SqlStdOperatorTable.IS_FALSE, node);
   }
 
   protected RexNode isNotFalse(RexNode node) {
+    assert node.getType().getSqlTypeName() == SqlTypeName.BOOLEAN;
     return rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_FALSE, node);
   }
 
   protected RexNode isTrue(RexNode node) {
+    assert node.getType().getSqlTypeName() == SqlTypeName.BOOLEAN;
     return rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, node);
   }
 
   protected RexNode isNotTrue(RexNode node) {
+    assert node.getType().getSqlTypeName() == SqlTypeName.BOOLEAN;
     return rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_TRUE, node);
   }
 
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 4cd61fe..1a8a748 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
@@ -931,15 +931,17 @@ class RexProgramTest extends RexProgramTestBase {
     checkSimplify(rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, aRef),
         "true");
 
-    // condition, and the inverse - nothing to do due to null values
-    checkSimplify2(and(le(hRef, literal(1)), gt(hRef, literal(1))),
-        "AND(<=(?0.h, 1), >(?0.h, 1))",
+    // condition, and the inverse
+    checkSimplify3(and(le(hRef, literal(1)), gt(hRef, literal(1))),
+        "<>(?0.h, ?0.h)",
+        "false",
         "false");
 
     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))",
+    checkSimplify3(and(lt(hRef, literal(1)), eq(hRef, literal(1)), ge(hRef, literal(1))),
+        "<>(?0.h, ?0.h)",
+        "false",
         "false");
 
     checkSimplify(and(lt(hRef, literal(1)), or(falseLiteral, falseLiteral)),
@@ -1404,16 +1406,19 @@ class RexProgramTest extends RexProgramTestBase {
 
   @Test void testSimplifyOrTerms() {
     final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
+    final RelDataType boolType = typeFactory.createSqlType(SqlTypeName.BOOLEAN);
     final RelDataType rowType = typeFactory.builder()
         .add("a", intType).nullable(false)
         .add("b", intType).nullable(true)
         .add("c", intType).nullable(true)
+        .add("d", boolType).nullable(true)
         .build();
 
     final RexDynamicParam range = rexBuilder.makeDynamicParam(rowType, 0);
     final RexNode aRef = rexBuilder.makeFieldAccess(range, 0);
     final RexNode bRef = rexBuilder.makeFieldAccess(range, 1);
     final RexNode cRef = rexBuilder.makeFieldAccess(range, 2);
+    final RexNode dRef = rexBuilder.makeFieldAccess(range, 3);
     final RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
     final RexLiteral literal2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(2));
     final RexLiteral literal3 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(3));
@@ -1495,11 +1500,11 @@ class RexProgramTest extends RexProgramTestBase {
     // Careful of the excluded middle!
     // We cannot simplify "b <> 1 or b = 1" to "true" because if b is null, the
     // result is unknown.
-    // TODO: "b is not unknown" would be the best simplification.
+    // TODO: "b = b" would be the best simplification.
     final RexNode simplified =
         this.simplify.simplifyUnknownAs(neOrEq, RexUnknownAs.UNKNOWN);
     assertThat(simplified.toString(),
-        equalTo("OR(<>(?0.b, 1), =(?0.b, 1))"));
+        equalTo("OR(IS NOT NULL(?0.b), null)"));
 
     // "a is null or a is not null" ==> "true"
     checkSimplifyFilter(
@@ -1554,12 +1559,12 @@ class RexProgramTest extends RexProgramTestBase {
             isNull(cRef)),
         "OR(IS NULL(?0.c), IS NOT NULL(?0.b))");
 
-    // "b is null or b is not false" => "b is null or b"
-    // (because after the first term we know that b cannot be null)
+    // "d is null or d is not false" => "d is null or d"
+    // (because after the first term we know that d cannot be null)
     checkSimplifyFilter(
-        or(isNull(bRef),
-            isNotFalse(bRef)),
-        "OR(IS NULL(?0.b), ?0.b)");
+        or(isNull(dRef),
+            isNotFalse(dRef)),
+        "OR(IS NULL(?0.d), ?0.d)");
 
     // multiple predicates are handled correctly
     checkSimplifyFilter(
@@ -1603,7 +1608,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) OR NULL])")
+    checkSimplify(expr, "SEARCH($0, Sarg[[15..+\u221e); NULL AS TRUE])")
         .expandedSearch("OR(IS NULL($0), >=($0, 15))");
   }
 
@@ -1622,7 +1627,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) OR NULL])";
+        "SEARCH($0, Sarg[(0..12), [15..+\u221e); NULL AS TRUE])";
     final String expanded =
         "OR(IS NULL($0), AND(>($0, 0), <($0, 12)), >=($0, 15))";
     checkSimplify(expr, simplified)
@@ -1651,7 +1656,7 @@ class RexProgramTest extends RexProgramTestBase {
                 eq(aRef, literal(5)))),
         isNull(aRef));
     final String simplified =
-        "SEARCH($0, Sarg[(-\u221e..3), (3..5), (5..+\u221e) OR NULL])";
+        "SEARCH($0, Sarg[(-\u221e..3), (3..5), (5..+\u221e); NULL AS TRUE])";
     final String expanded = "OR(IS NULL($0), AND(<>($0, 3), <>($0, 5)))";
     checkSimplify(expr, simplified)
         .expandedSearch(expanded);
@@ -1679,8 +1684,8 @@ class RexProgramTest extends RexProgramTestBase {
         isNotNull(aRef),
         gt(aRef, literal(3)),
         lt(aRef, literal(10)));
-    final String simplified = "SEARCH($0, Sarg[(3..10)])";
-    final String expanded = "AND(>($0, 3), <($0, 10))";
+    final String simplified = "SEARCH($0, Sarg[(3..10); NULL AS FALSE])";
+    final String expanded = "AND(IS NOT NULL($0), AND(>($0, 3), <($0, 10)))";
     checkSimplify(expr, simplified)
         .expandedSearch(expanded);
   }
@@ -1761,13 +1766,13 @@ class RexProgramTest extends RexProgramTestBase {
   @Test void testSimplifyEqOrIsNullAndEq() {
     // (deptno = 20 OR deptno IS NULL) AND deptno = 10
     //   ==>
-    // false
+    // deptno <> deptno
     final RexNode e =
         and(
             or(eq(vInt(), literal(20)),
                 isNull(vInt())),
         eq(vInt(), literal(10)));
-    checkSimplify(e, "false");
+    checkSimplify3(e, "<>(?0.int0, ?0.int0)", "false", "IS NULL(?0.int0)");
   }
 
   @Test void testSimplifyEqOrIsNullAndEqSame() {
@@ -1795,12 +1800,13 @@ class RexProgramTest extends RexProgramTestBase {
     // deptno in (20, 10) and deptno = 30
     //   ==>
     // false
-    checkSimplify2(
+    checkSimplify3(
         and(
         in(vInt(), literal(20), literal(10)),
         eq(vInt(), literal(30))),
-        "AND(SEARCH(?0.int0, Sarg[10, 20]), =(?0.int0, 30))",
-        "false");
+        "<>(?0.int0, ?0.int0)",
+        "false",
+        "IS NULL(?0.int0)");
   }
 
   @Test void testSimplifyInOr() {
@@ -1816,26 +1822,24 @@ class RexProgramTest extends RexProgramTestBase {
 
   /** Test strategies for {@code SargCollector.canMerge(Sarg, RexUnknownAs)}. */
   @Test void testSargMerge() {
-    checkSimplify2(
-        or(
-            ne(vInt(), literal(1)),
+    checkSimplify3(
+        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)),
+        "OR(IS NOT NULL(?0.int0), null)",
+        "IS NOT NULL(?0.int0)",
+        "true");
+    checkSimplify3(
+        and(gt(vInt(), literal(5)),
             lt(vInt(), literal(3))),
-        "AND(>(?0.int0, 5), <(?0.int0, 3))",
-        "false");
+        "<>(?0.int0, ?0.int0)",
+        "false",
+        "IS NULL(?0.int0)");
     checkSimplify(
-        or(
-            falseLiteral,
+        or(falseLiteral,
             isNull(vInt())),
         "IS NULL(?0.int0)");
     checkSimplify(
-        and(
-            trueLiteral,
+        and(trueLiteral,
             isNotNull(vInt())),
         "IS NOT NULL(?0.int0)");
   }
@@ -2779,12 +2783,11 @@ class RexProgramTest extends RexProgramTestBase {
   }
 
   @Test void testSimplifyOrIsNull() {
+    String expected = "SEARCH(?0.int0, Sarg[10; NULL AS TRUE])";
     // x = 10 OR x IS NULL
-    checkSimplify(or(eq(vInt(0), literal(10)), isNull(vInt(0))),
-        "SEARCH(?0.int0, Sarg[10 OR NULL])");
+    checkSimplify(or(eq(vInt(0), literal(10)), isNull(vInt(0))), expected);
     // 10 = x OR x IS NULL
-    checkSimplify(or(eq(literal(10), vInt(0)), isNull(vInt(0))),
-        "SEARCH(?0.int0, Sarg[10 OR NULL])");
+    checkSimplify(or(eq(literal(10), vInt(0)), isNull(vInt(0))), expected);
   }
 
   @Test void testSimplifyOrNot() {
@@ -2819,49 +2822,52 @@ class RexProgramTest extends RexProgramTestBase {
   @SuppressWarnings("UnstableApiUsage")
   @Test void testSargComplexity() {
     checkSarg("complexity of 'x is not null'",
-        Sarg.of(false, RangeSets.<Integer>rangeSetAll()),
-        is(1), is("Sarg[NOT NULL]"));
+        Sarg.of(RexUnknownAs.FALSE, RangeSets.<Integer>rangeSetAll()),
+        is(1), is("Sarg[IS NOT NULL]"));
     checkSarg("complexity of 'x is null'",
-        Sarg.of(true, ImmutableRangeSet.<Integer>of()),
-        is(1), is("Sarg[NULL]"));
+        Sarg.of(RexUnknownAs.TRUE, ImmutableRangeSet.<Integer>of()),
+        is(1), is("Sarg[IS NULL]"));
     checkSarg("complexity of 'false'",
-        Sarg.of(false, ImmutableRangeSet.<Integer>of()),
+        Sarg.of(RexUnknownAs.FALSE, ImmutableRangeSet.<Integer>of()),
         is(0), is("Sarg[FALSE]"));
     checkSarg("complexity of 'true'",
-        Sarg.of(true, RangeSets.<Integer>rangeSetAll()),
+        Sarg.of(RexUnknownAs.TRUE, RangeSets.<Integer>rangeSetAll()),
         is(2), is("Sarg[TRUE]"));
 
     checkSarg("complexity of 'x = 1'",
-        Sarg.of(false, ImmutableRangeSet.of(Range.singleton(1))),
+        Sarg.of(RexUnknownAs.UNKNOWN, ImmutableRangeSet.of(Range.singleton(1))),
         is(1), is("Sarg[1]"));
     checkSarg("complexity of 'x > 1'",
-        Sarg.of(false, ImmutableRangeSet.of(Range.greaterThan(1))),
+        Sarg.of(RexUnknownAs.UNKNOWN,
+            ImmutableRangeSet.of(Range.greaterThan(1))),
         is(1), is("Sarg[(1..+\u221E)]"));
     checkSarg("complexity of 'x >= 1'",
-        Sarg.of(false, ImmutableRangeSet.of(Range.atLeast(1))),
+        Sarg.of(RexUnknownAs.UNKNOWN, 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]"));
+        Sarg.of(RexUnknownAs.TRUE, ImmutableRangeSet.of(Range.greaterThan(1))),
+        is(2), is("Sarg[(1..+\u221E); NULL AS TRUE]"));
     checkSarg("complexity of 'x <> 1'",
-        Sarg.of(false, ImmutableRangeSet.of(Range.singleton(1)).complement()),
+        Sarg.of(RexUnknownAs.UNKNOWN,
+            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]"));
+        Sarg.of(RexUnknownAs.TRUE,
+            ImmutableRangeSet.of(Range.singleton(1)).complement()),
+        is(2), is("Sarg[(-\u221E..1), (1..+\u221E); NULL AS TRUE]"));
     checkSarg("complexity of 'x < 10 or x >= 20'",
-        Sarg.of(false,
+        Sarg.of(RexUnknownAs.UNKNOWN,
             ImmutableRangeSet.copyOf(
                 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,
+        Sarg.of(RexUnknownAs.UNKNOWN,
             ImmutableRangeSet.copyOf(
                 Arrays.asList(Range.singleton(2), Range.singleton(4),
                     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,
+        Sarg.of(RexUnknownAs.UNKNOWN,
             ImmutableRangeSet.copyOf(
                 Arrays.asList(Range.closed(3, 8),
                     Range.closed(10, 20)))),
@@ -2886,7 +2892,7 @@ class RexProgramTest extends RexProgramTestBase {
   }
 
   @Test void testIsNullRecursion() {
-    // make sure that simplifcation is visiting below isX expressions
+    // make sure that simplification is visiting below isX expressions
     checkSimplify(
         isNull(or(coalesce(nullBool, trueLiteral), falseLiteral)),
         "false");
@@ -3078,8 +3084,14 @@ class RexProgramTest extends RexProgramTestBase {
    * RexSimplify should simplify more always true OR expressions</a>. */
   @Test void testSimplifyLike() {
     final RexNode ref = input(tVarchar(true, 10), 0);
-    checkSimplify(like(ref, literal("%")), "true");
-    checkSimplify(like(ref, literal("%"), literal("#")), "true");
+    checkSimplify(like(ref, literal("%")),
+        "OR(null, IS NOT NULL($0))");
+    checkSimplify(like(ref, literal("%"), literal("#")),
+        "OR(null, IS NOT NULL($0))");
+    checkSimplify(or(isNull(ref), like(ref, literal("%"))),
+        "true");
+    checkSimplify(or(isNull(ref), like(ref, literal("%"), literal("#"))),
+        "true");
     checkSimplifyUnchanged(like(ref, literal("%A")));
     checkSimplifyUnchanged(like(ref, literal("%A"), literal("#")));
   }
diff --git a/core/src/test/java/org/apache/calcite/rex/RexProgramTestBase.java b/core/src/test/java/org/apache/calcite/rex/RexProgramTestBase.java
index 0edf0c3..36820db 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramTestBase.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramTestBase.java
@@ -109,7 +109,7 @@ class RexProgramTestBase extends RexProgramBuilderBase {
    *     as false
    */
   protected void checkSimplify2(RexNode node, String expected,
-                              String expectedFalse) {
+      String expectedFalse) {
     checkSimplify3_(node, expected, expectedFalse, expected);
     if (expected.equals(expectedFalse)) {
       throw new AssertionError("expected == expectedFalse; use checkSimplify");
@@ -117,7 +117,7 @@ class RexProgramTestBase extends RexProgramBuilderBase {
   }
 
   protected void checkSimplify3(RexNode node, String expected,
-                              String expectedFalse, String expectedTrue) {
+      String expectedFalse, String expectedTrue) {
     checkSimplify3_(node, expected, expectedFalse, expectedTrue);
     if (expected.equals(expectedFalse) && expected.equals(expectedTrue)) {
       throw new AssertionError("expected == expectedFalse == expectedTrue; "
@@ -131,18 +131,10 @@ class RexProgramTestBase extends RexProgramBuilderBase {
   protected SimplifiedNode checkSimplify3_(RexNode node, String expected,
       String expectedFalse, String expectedTrue) {
     final RexNode simplified =
-        simplify.simplifyUnknownAs(node, RexUnknownAs.UNKNOWN);
-    assertThat("simplify(unknown as unknown): " + node,
-        simplified.toString(), equalTo(expected));
+        checkSimplifyAs(node, RexUnknownAs.UNKNOWN, is(expected));
     if (node.getType().getSqlTypeName() == SqlTypeName.BOOLEAN) {
-      final RexNode simplified2 =
-          simplify.simplifyUnknownAs(node, RexUnknownAs.FALSE);
-      assertThat("simplify(unknown as false): " + node,
-          simplified2.toString(), equalTo(expectedFalse));
-      final RexNode simplified3 =
-          simplify.simplifyUnknownAs(node, RexUnknownAs.TRUE);
-      assertThat("simplify(unknown as true): " + node,
-          simplified3.toString(), equalTo(expectedTrue));
+      checkSimplifyAs(node, RexUnknownAs.FALSE, is(expectedFalse));
+      checkSimplifyAs(node, RexUnknownAs.TRUE, is(expectedTrue));
     } else {
       assertThat("node type is not BOOLEAN, so <<expectedFalse>> should match <<expected>>",
           expectedFalse, is(expected));
@@ -152,15 +144,21 @@ class RexProgramTestBase extends RexProgramBuilderBase {
     return new SimplifiedNode(rexBuilder, node, simplified);
   }
 
-  protected Node checkSimplifyFilter(RexNode node, String expected) {
+  private RexNode checkSimplifyAs(RexNode node, RexUnknownAs unknownAs,
+      Matcher<String> matcher) {
     final RexNode simplified =
-        this.simplify.simplifyUnknownAs(node, RexUnknownAs.FALSE);
-    assertThat(simplified.toString(), equalTo(expected));
-    return node(node);
+        simplify.simplifyUnknownAs(node, unknownAs);
+    assertThat(("simplify(unknown as " + unknownAs + "): ") + node,
+        simplified.toString(), matcher);
+    return simplified;
+  }
+
+  protected void checkSimplifyFilter(RexNode node, String expected) {
+    checkSimplifyAs(node, RexUnknownAs.FALSE, is(expected));
   }
 
-  protected void checkSimplifyFilter(RexNode node, RelOptPredicateList predicates,
-                                   String expected) {
+  protected void checkSimplifyFilter(RexNode node,
+      RelOptPredicateList predicates, String expected) {
     final RexNode simplified =
         simplify.withPredicates(predicates)
             .simplifyUnknownAs(node, RexUnknownAs.FALSE);
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 c822ca7..b083332 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -683,7 +683,9 @@ public class RelBuilderTest {
                 builder.or(
                     builder.equals(builder.field("DEPTNO"),
                         builder.literal(20)),
-                    builder.and(builder.literal(null),
+                    builder.and(
+                        builder.cast(builder.literal(null),
+                            SqlTypeName.BOOLEAN),
                         builder.equals(builder.field("DEPTNO"),
                             builder.literal(10)),
                         builder.and(builder.isNull(builder.field(6)),
@@ -700,7 +702,7 @@ public class RelBuilderTest {
             .build();
     final String expected = ""
         + "LogicalProject(DEPTNO=[$7], COMM=[CAST($6):SMALLINT NOT NULL], "
-        + "$f2=[OR(SEARCH($7, Sarg[20, 30]), AND(null:NULL, =($7, 10), "
+        + "$f2=[OR(SEARCH($7, Sarg[20, 30]), AND(null, =($7, 10), "
         + "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";
diff --git a/core/src/test/java/org/apache/calcite/test/fuzzer/RexFuzzer.java b/core/src/test/java/org/apache/calcite/test/fuzzer/RexFuzzer.java
index 9b8d1fc..3347c46 100644
--- a/core/src/test/java/org/apache/calcite/test/fuzzer/RexFuzzer.java
+++ b/core/src/test/java/org/apache/calcite/test/fuzzer/RexFuzzer.java
@@ -21,6 +21,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexProgramBuilderBase;
+import org.apache.calcite.rex.RexUnknownAs;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -263,6 +264,8 @@ public class RexFuzzer extends RexProgramBuilderBase {
   public RexNode fuzzSearch(Random r, RexNode intExpression) {
     final RangeSet<BigDecimal> rangeSet = TreeRangeSet.create();
     final Generator<BigDecimal> integerGenerator = RexFuzzer::fuzzInt;
+    final Generator<RexUnknownAs> unknownGenerator =
+        enumGenerator(RexUnknownAs.class);
     int i = 0;
     for (;;) {
       rangeSet.add(fuzzRange(r, integerGenerator));
@@ -270,12 +273,16 @@ public class RexFuzzer extends RexProgramBuilderBase {
         break;
       }
     }
-    final boolean containsNull = r.nextInt(3) == 0;
-    final Sarg<BigDecimal> sarg = Sarg.of(containsNull, rangeSet);
-    final RexNode c = rexBuilder.makeCall(SqlStdOperatorTable.SEARCH, intExpression,
+    final Sarg<BigDecimal> sarg =
+        Sarg.of(unknownGenerator.generate(r), rangeSet);
+    return rexBuilder.makeCall(SqlStdOperatorTable.SEARCH, intExpression,
         rexBuilder.makeSearchArgumentLiteral(sarg, intExpression.getType()));
-    System.out.println(c);
-    return c;
+  }
+
+  private static <T extends Enum<T>> Generator<T> enumGenerator(
+      Class<T> enumClass) {
+    final T[] enumConstants = enumClass.getEnumConstants();
+    return r -> enumConstants[r.nextInt(enumConstants.length)];
   }
 
   <T extends Comparable<T>> Range<T> fuzzRange(Random r,
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 44b1143..30b4367 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -11743,7 +11743,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..+∞)]), 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, <>($9, 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/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java b/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java
index 65ab344..9767b73 100644
--- a/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java
+++ b/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java
@@ -53,8 +53,8 @@ class DruidDateRangeRulesTest {
     final Fixture2 f = new Fixture2();
     checkDateRange(f,
         f.and(
-            f.le(f.timestampLiteral(2011, Calendar.JANUARY, 1), f.t),
-            f.le(f.t, f.timestampLiteral(2012, Calendar.FEBRUARY, 2))),
+            f.le(f.timestampLiteral(2011, Calendar.JANUARY, 1), f.ts),
+            f.le(f.ts, f.timestampLiteral(2012, Calendar.FEBRUARY, 2))),
         is("[2011-01-01T00:00:00.000Z/2012-02-02T00:00:00.001Z]"));
   }