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/08/20 21:15:48 UTC

[calcite] branch master updated (6968f0a -> 9ca61f0)

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 6968f0a  Add assertion that relnode after pass through traits has the same convention
     new dc7bcc5  Refactor
     new ca6c590  [CALCITE-4170] Improve simplification of "<>" predicates
     new 9ca61f0  [CALCITE-4159] Simplify always-true expressions (such as LIKE '%') to TRUE

The 3 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/plan/RelOptPredicateList.java   |  46 ++
 .../calcite/rel/RelReferentialConstraint.java      |  11 +-
 .../calcite/rel/RelReferentialConstraintImpl.java  |   4 -
 .../org/apache/calcite/rel/core/RelFactories.java  |   7 +-
 .../rules/materialize/MaterializedViewRule.java    |  28 +-
 .../java/org/apache/calcite/rex/RexSimplify.java   | 216 ++++++---
 .../main/java/org/apache/calcite/sql/SqlKind.java  |   2 +-
 .../sql/validate/implicit/TypeCoercionImpl.java    |   6 +-
 .../java/org/apache/calcite/tools/RelBuilder.java  |   6 +-
 .../java/org/apache/calcite/util/RangeSets.java    |  46 ++
 .../main/java/org/apache/calcite/util/Util.java    |  39 ++
 .../apache/calcite/rex/RexProgramBuilderBase.java  |  29 +-
 .../org/apache/calcite/rex/RexProgramTest.java     | 510 ++++++++++++---------
 .../apache/calcite/test/SqlToRelConverterTest.java |   4 +-
 .../java/org/apache/calcite/util/UtilTest.java     |  65 +++
 .../apache/calcite/test/SqlToRelConverterTest.xml  |  13 +-
 core/src/test/resources/sql/some.iq                |   2 +-
 17 files changed, 705 insertions(+), 329 deletions(-)
 create mode 100644 core/src/main/java/org/apache/calcite/util/RangeSets.java


[calcite] 03/03: [CALCITE-4159] Simplify always-true expressions (such as LIKE '%') to TRUE

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 9ca61f020d134032dac8167b83bf431f078e33c5
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Sun Aug 9 11:43:18 2020 -0700

    [CALCITE-4159] Simplify always-true expressions (such as LIKE '%') to TRUE
    
    RexSimplify needs to recognize that the following expressions
    can be simplified to TRUE:
    
    * c < 0 OR c >= 0 OR c IS NULL
    * c < 2 OR c > 0 OR c IS NULL
    * c LIKE '%' OR c IS NULL
    
    To enable this fix, we pull 'IS NULL' terms to the front of a list of
    OR-terms, so that by the time the subsequent terms are reached they
    know that the term's value is not NULL.
    
    Close apache/calcite#2105
---
 .../apache/calcite/plan/RelOptPredicateList.java   |  46 ++++++
 .../java/org/apache/calcite/rex/RexSimplify.java   | 179 ++++++++++++++++-----
 .../main/java/org/apache/calcite/util/Util.java    |  39 +++++
 .../apache/calcite/rex/RexProgramBuilderBase.java  |   8 +
 .../org/apache/calcite/rex/RexProgramTest.java     | 103 ++++++++++--
 .../java/org/apache/calcite/util/UtilTest.java     |  23 +++
 6 files changed, 348 insertions(+), 50 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java b/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java
index ab20971..b38b135 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java
@@ -17,12 +17,16 @@
 package org.apache.calcite.plan;
 
 import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlKind;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
+import java.util.Collection;
 import java.util.Objects;
 
 /**
@@ -148,6 +152,25 @@ public class RelOptPredicateList {
         leftInferredPredicateList, rightInferredPredicatesList, constantMap);
   }
 
+  @Override public String toString() {
+    final StringBuilder b = new StringBuilder("{");
+    append(b, "pulled", pulledUpPredicates);
+    append(b, "left", leftInferredPredicates);
+    append(b, "right", rightInferredPredicates);
+    append(b, "constants", constantMap.entrySet());
+    return b.append("}").toString();
+  }
+
+  private static void append(StringBuilder b, String key, Collection<?> value) {
+    if (!value.isEmpty()) {
+      if (b.length() > 1) {
+        b.append(", ");
+      }
+      b.append(key);
+      b.append(value);
+    }
+  }
+
   public RelOptPredicateList union(RexBuilder rexBuilder,
       RelOptPredicateList list) {
     if (this == EMPTY) {
@@ -180,4 +203,27 @@ public class RelOptPredicateList {
         RexUtil.shift(leftInferredPredicates, offset),
         RexUtil.shift(rightInferredPredicates, offset));
   }
+
+  /** Returns whether an expression is effectively NOT NULL due to an
+   * {@code e IS NOT NULL} condition in this predicate list. */
+  public boolean isEffectivelyNotNull(RexNode e) {
+    if (!e.getType().isNullable()) {
+      return true;
+    }
+    for (RexNode p : pulledUpPredicates) {
+      if (p.getKind() == SqlKind.IS_NOT_NULL
+          && ((RexCall) p).getOperands().get(0).equals(e)) {
+        return true;
+      }
+    }
+    if (SqlKind.COMPARISON.contains(e.getKind())) {
+      // A comparison with a literal, such as 'ref < 10', is not null if 'ref'
+      // is not null.
+      RexCall call = (RexCall) e;
+      if (call.getOperands().get(1) instanceof RexLiteral) {
+        return isEffectivelyNotNull(call.getOperands().get(0));
+      }
+    }
+    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 ee99936..5e92098 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -38,10 +38,13 @@ import org.apache.calcite.util.Util;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.BoundType;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableRangeSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Range;
+import com.google.common.collect.RangeSet;
 import com.google.common.collect.Sets;
+import com.google.common.collect.TreeRangeSet;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -292,6 +295,8 @@ public class RexSimplify {
     case LESS_THAN_OR_EQUAL:
     case NOT_EQUALS:
       return simplifyComparison((RexCall) e, unknownAs);
+    case LIKE:
+      return simplifyLike((RexCall) e);
     case MINUS_PREFIX:
       return simplifyUnaryMinus((RexCall) e, unknownAs);
     case PLUS_PREFIX:
@@ -317,6 +322,16 @@ public class RexSimplify {
     return rexBuilder.makeCall(e.getType(), e.getOperator(), operands);
   }
 
+  private RexNode simplifyLike(RexCall e) {
+    if (e.operands.get(1) instanceof RexLiteral) {
+      final RexLiteral literal = (RexLiteral) e.operands.get(1);
+      if (literal.getValueAs(String.class).equals("%")) {
+        return rexBuilder.makeLiteral(true);
+      }
+    }
+    return simplifyGenericNode(e);
+  }
+
   // e must be a comparison (=, >, >=, <, <=, !=)
   private RexNode simplifyComparison(RexCall e, RexUnknownAs unknownAs) {
     //noinspection unchecked
@@ -513,7 +528,7 @@ public class RexSimplify {
     RexSimplify simplify = this;
     for (int i = 0; i < terms.size(); i++) {
       final RexNode t = terms.get(i);
-      if (!allowedAsPredicateDuringOrSimplification(t)) {
+      if (!simplify.allowedAsPredicateDuringOrSimplification(t)) {
         continue;
       }
       final RexNode t2 = simplify.simplify(t, unknownAs);
@@ -540,14 +555,7 @@ public class RexSimplify {
    */
   private boolean allowedAsPredicateDuringOrSimplification(final RexNode t) {
     Predicate predicate = Predicate.of(t);
-    if (predicate == null) {
-      return false;
-    }
-    SqlKind kind = t.getKind();
-    if (SqlKind.COMPARISON.contains(kind) && t.getType().isNullable()) {
-      return false;
-    }
-    return true;
+    return predicate != null && predicate.allowedInOr(predicates);
   }
 
   private RexNode simplifyNot(RexCall call, RexUnknownAs unknownAs) {
@@ -711,7 +719,7 @@ public class RexSimplify {
     case IS_NOT_FALSE:
       // x IS TRUE ==> x (if x is not nullable)
       // x IS NOT FALSE ==> x (if x is not nullable)
-      if (!a.getType().isNullable()) {
+      if (predicates.isEffectivelyNotNull(a)) {
         return simplify(a, unknownAs);
       } else {
         RexNode newSub =
@@ -725,7 +733,7 @@ public class RexSimplify {
     case IS_NOT_TRUE:
       // x IS NOT TRUE ==> NOT x (if x is not nullable)
       // x IS FALSE ==> NOT x (if x is not nullable)
-      if (!a.getType().isNullable()) {
+      if (predicates.isEffectivelyNotNull(a)) {
         return simplify(rexBuilder.makeCall(SqlStdOperatorTable.NOT, a), unknownAs);
       }
       break;
@@ -1592,32 +1600,31 @@ public class RexSimplify {
     }
 
     final C v0 = comparison.literal.getValueAs(clazz);
-    final Range<C> range = range(comparison.kind, v0);
-    final Range<C> range2 =
-        residue(comparison.ref, range, predicates.pulledUpPredicates,
+    final RangeSet<C> rangeSet = rangeSet(comparison.kind, v0);
+    final RangeSet<C> rangeSet2 =
+        residue(comparison.ref, rangeSet, predicates.pulledUpPredicates,
             clazz);
-    if (range2 == null) {
+    if (rangeSet2.isEmpty()) {
       // Term is impossible to satisfy given these predicates
       return rexBuilder.makeLiteral(false);
-    } else if (range2.equals(range)) {
+    } else if (rangeSet2.equals(rangeSet)) {
       // no change
       return e;
-    } else if (range2.equals(Range.all())) {
+    } else if (rangeSet2.equals(RangeSets.rangeSetAll())) {
       // Range is always satisfied given these predicates; but nullability might
       // be problematic
       return simplify(
           rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, comparison.ref),
           RexUnknownAs.UNKNOWN);
-    } else if (range2.lowerEndpoint().equals(range2.upperEndpoint())) {
-      if (range2.lowerBoundType() == BoundType.OPEN
-          || range2.upperBoundType() == BoundType.OPEN) {
-        // range is a point, but does not include its endpoint, therefore is
-        // effectively empty
-        return rexBuilder.makeLiteral(false);
-      }
+    } else if (rangeSet2.asRanges().size() == 1
+        && Iterables.getOnlyElement(rangeSet2.asRanges()).hasLowerBound()
+        && Iterables.getOnlyElement(rangeSet2.asRanges()).hasUpperBound()
+        && Iterables.getOnlyElement(rangeSet2.asRanges()).lowerEndpoint()
+        .equals(Iterables.getOnlyElement(rangeSet2.asRanges()).upperEndpoint())) {
+      final Range<C> r = Iterables.getOnlyElement(rangeSet2.asRanges());
       // range is now a point; it's worth simplifying
       return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, comparison.ref,
-          rexBuilder.makeLiteral(range2.lowerEndpoint(),
+          rexBuilder.makeLiteral(r.lowerEndpoint(),
               comparison.literal.getType(), comparison.literal.getTypeName()));
     } else {
       // range has been reduced but it's not worth simplifying
@@ -1638,12 +1645,13 @@ public class RexSimplify {
    * <li>{@code residue($0 < 10, [$0 < 20, $0 > 0])} returns {@code $0 < 10}
    * </ul>
    */
-  private <C extends Comparable<C>> Range<C> residue(RexNode ref, Range<C> r0,
-      List<RexNode> predicates, Class<C> clazz) {
-    Range<C> result = r0;
+  private <C extends Comparable<C>> RangeSet<C> residue(RexNode ref,
+      RangeSet<C> r0, List<RexNode> predicates, Class<C> clazz) {
+    RangeSet<C> result = r0;
     for (RexNode predicate : predicates) {
       switch (predicate.getKind()) {
       case EQUALS:
+      case NOT_EQUALS:
       case LESS_THAN:
       case LESS_THAN_OR_EQUAL:
       case GREATER_THAN:
@@ -1653,25 +1661,83 @@ public class RexSimplify {
             && call.operands.get(1) instanceof RexLiteral) {
           final RexLiteral literal = (RexLiteral) call.operands.get(1);
           final C c1 = literal.getValueAs(clazz);
-          final Range<C> r1 = range(predicate.getKind(), c1);
-          if (result.encloses(r1)) {
-            // Given these predicates, term is always satisfied.
-            // e.g. r0 is "$0 < 10", r1 is "$0 < 5"
-            result = Range.all();
-            continue;
+          switch (predicate.getKind()) {
+          case NOT_EQUALS:
+            // We want to intersect result with the range set of everything but
+            // c1. We subtract the point c1 from result, which is equivalent.
+            final Range<C> pointRange = range(SqlKind.EQUALS, c1);
+            final RangeSet<C> notEqualsRangeSet =
+                ImmutableRangeSet.of(pointRange).complement();
+            if (result.enclosesAll(notEqualsRangeSet)) {
+              result = RangeSets.rangeSetAll();
+              continue;
+            }
+            result = RangeSets.minus(result, pointRange);
+            break;
+          default:
+            final Range<C> r1 = range(predicate.getKind(), c1);
+            if (result.encloses(r1)) {
+              // Given these predicates, term is always satisfied.
+              // e.g. r0 is "$0 < 10", r1 is "$0 < 5"
+              result = RangeSets.rangeSetAll();
+              continue;
+            }
+            result = result.subRangeSet(r1);
           }
-          if (result.isConnected(r1)) {
-            result = result.intersection(r1);
-            continue;
+          if (result.isEmpty()) {
+            break; // short-cut
           }
-          // Ranges do not intersect. Return null meaning the empty range.
-          return null;
         }
       }
     }
     return result;
   }
 
+  /** Subtracts a range from a range set. */
+  public static <C extends Comparable<C>> RangeSet<C> minus(RangeSet<C> rangeSet,
+      Range<C> range) {
+    final TreeRangeSet<C> mutableRangeSet = TreeRangeSet.create(rangeSet);
+    mutableRangeSet.remove(range);
+    return mutableRangeSet.equals(rangeSet) ? rangeSet
+        : ImmutableRangeSet.copyOf(mutableRangeSet);
+  }
+
+  // TODO
+  public static <C extends Comparable<C>> RangeSet<C> minus0(RangeSet<C> rangeSet,
+      Range<C> range) {
+    ImmutableRangeSet<C> x = ImmutableRangeSet.of(range).complement();
+    final ImmutableRangeSet.Builder<C> builder = ImmutableRangeSet.<C>builder();
+    for (Range<C> r : x.asRanges()) {
+      builder.addAll(rangeSet.subRangeSet(r));
+    }
+    return builder.build();
+  }
+
+  /** Returns the intersection of two range sets.
+   *
+   * <p>For example:
+   * <ul>
+   *   <li>empty intersect empty &rarr; empty
+   *   <li>{1, [3, 5], (9, &inf;)} intersect {1, 3, 5, 7, 9, 11}} &rarr;
+   *   {1, 3, 5, 11}
+   *   <li>all intersect {[3, 5)} &rarr; {[3, 5)}
+   * </ul>
+   *
+   * @param s0 First range set
+   * @param s1 Second range set
+   * @param <C> Value type
+   * @return Intersection of range sets
+   */
+  // TODO
+  private static <C extends Comparable<C>> RangeSet<C> intersect(RangeSet<C> s0,
+      RangeSet<C> s1) {
+    RangeSet<C> s = s0;
+    for (Range<C> range : s1.asRanges()) {
+      s = s.subRangeSet(range);
+    }
+    return s;
+  }
+
   /** Simplifies OR(x, x) into x, and similar.
    * The simplified expression returns UNKNOWN values as is (not as FALSE). */
   @Deprecated // to be removed before 2.0
@@ -1682,10 +1748,13 @@ public class RexSimplify {
 
   private RexNode simplifyOr(RexCall call, RexUnknownAs unknownAs) {
     assert call.getKind() == SqlKind.OR;
-    final List<RexNode> terms = RelOptUtil.disjunctions(call);
+    final List<RexNode> terms0 = RelOptUtil.disjunctions(call);
+    final List<RexNode> terms;
     if (predicateElimination) {
+      terms = Util.moveToHead(terms0, e -> e.getKind() == SqlKind.IS_NULL);
       simplifyOrTerms(terms, unknownAs);
     } else {
+      terms = terms0;
       simplifyList(terms, unknownAs);
     }
     return simplifyOrs(terms, unknownAs);
@@ -2239,6 +2308,25 @@ public class RexSimplify {
     }
   }
 
+  private static <C extends Comparable<C>> RangeSet<C> rangeSet(SqlKind comparison,
+      C c) {
+    switch (comparison) {
+    case EQUALS:
+    case LESS_THAN:
+    case LESS_THAN_OR_EQUAL:
+    case GREATER_THAN:
+    case GREATER_THAN_OR_EQUAL:
+      return ImmutableRangeSet.of(range(comparison, c));
+    case NOT_EQUALS:
+      return ImmutableRangeSet.<C>builder()
+          .add(range(SqlKind.LESS_THAN, c))
+          .add(range(SqlKind.GREATER_THAN, c))
+          .build();
+    default:
+      throw new AssertionError();
+    }
+  }
+
   /** Marker interface for predicates (expressions that evaluate to BOOLEAN). */
   private interface Predicate {
     /** Wraps an expression in a Predicate or returns null. */
@@ -2249,6 +2337,12 @@ public class RexSimplify {
       }
       return IsPredicate.of(t);
     }
+
+    /** Returns whether this predicate can be used while simplifying other OR
+     * operands. */
+    default boolean allowedInOr(RelOptPredicateList predicates) {
+      return true;
+    }
   }
 
   /** Represents a simple Comparison.
@@ -2300,6 +2394,11 @@ public class RexSimplify {
       }
       return null;
     }
+
+    @Override public boolean allowedInOr(RelOptPredicateList predicates) {
+      return !ref.getType().isNullable()
+          || predicates.isEffectivelyNotNull(ref);
+    }
   }
 
   /** Represents an IS Predicate. */
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index 1919341..7fbcb64 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -2435,6 +2435,45 @@ public class Util {
     return new FilteringIterator<>(iterator, predicate);
   }
 
+  /** Returns a list with any elements for which the predicate is true moved to
+   * the head of the list. The algorithm does not modify the list, is stable,
+   * and is idempotent. */
+  public static <E> List<E> moveToHead(List<E> terms, Predicate<E> predicate) {
+    if (alreadyAtFront(terms, predicate)) {
+      return terms;
+    }
+    final List<E> newTerms = new ArrayList<>(terms.size());
+    for (E term : terms) {
+      if (predicate.test(term)) {
+        newTerms.add(term);
+      }
+    }
+    for (E term : terms) {
+      if (!predicate.test(term)) {
+        newTerms.add(term);
+      }
+    }
+    return newTerms;
+  }
+
+  /** Returns whether of the elements of a list for which predicate is true
+   * occur before all elements where the predicate is false. (Returns true in
+   * corner cases such as empty list, all true, or all false. */
+  private static <E> boolean alreadyAtFront(List<E> list,
+      Predicate<E> predicate) {
+    boolean prev = true;
+    for (E e : list) {
+      final boolean pass = predicate.test(e);
+      if (pass && !prev) {
+        return false;
+      }
+      prev = pass;
+    }
+    return true;
+  }
+
+
+
   /** Returns a view of a list, picking the elements of a list with the given
    * set of ordinals. */
   public static <E> List<E> select(List<E> list, List<Integer> ordinals) {
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 0b60363..2d6fdc5 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramBuilderBase.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramBuilderBase.java
@@ -306,6 +306,14 @@ public abstract class RexProgramBuilderBase {
     return rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, n1, n2);
   }
 
+  protected RexNode like(RexNode ref, RexNode pattern) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.LIKE, ref, pattern);
+  }
+
+  protected RexNode like(RexNode ref, RexNode pattern, RexNode escape) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.LIKE, ref, pattern, escape);
+  }
+
   protected RexNode plus(RexNode n1, RexNode n2) {
     return rexBuilder.makeCall(SqlStdOperatorTable.PLUS, n1, n2);
   }
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 88db229..dfd145c 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
@@ -36,6 +36,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.RangeSets;
 import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.TimeString;
 import org.apache.calcite.util.TimestampString;
@@ -44,8 +45,11 @@ import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableRangeSet;
 import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Multimap;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeSet;
 
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
@@ -1342,6 +1346,44 @@ class RexProgramTest extends RexProgramTestBase {
         "false");
   }
 
+  @SuppressWarnings("UnstableApiUsage")
+  @Test void testRangeSetMinus() {
+    final RangeSet<Integer> setNone = ImmutableRangeSet.of();
+    final RangeSet<Integer> setAll = setNone.complement();
+    final RangeSet<Integer> setGt2 = ImmutableRangeSet.of(Range.greaterThan(2));
+    final RangeSet<Integer> setGt1 = ImmutableRangeSet.of(Range.greaterThan(1));
+    final RangeSet<Integer> setGe1 = ImmutableRangeSet.of(Range.atLeast(1));
+    final RangeSet<Integer> setGt0 = ImmutableRangeSet.of(Range.greaterThan(0));
+    final RangeSet<Integer> setComplex =
+        ImmutableRangeSet.<Integer>builder()
+            .add(Range.closed(0, 2))
+            .add(Range.singleton(3))
+            .add(Range.greaterThan(5))
+            .build();
+    assertThat(setComplex.toString(), is("[[0‥2], [3‥3], (5‥+∞)]"));
+
+    assertThat(RangeSets.minus(setAll, Range.singleton(1)).toString(),
+        is("[(-∞‥1), (1‥+∞)]"));
+    assertThat(RangeSets.minus(setNone, Range.singleton(1)), is(setNone));
+    assertThat(RangeSets.minus(setGt2, Range.singleton(1)), is(setGt2));
+    assertThat(RangeSets.minus(setGt1, Range.singleton(1)), is(setGt1));
+    assertThat(RangeSets.minus(setGe1, Range.singleton(1)), is(setGt1));
+    assertThat(RangeSets.minus(setGt0, Range.singleton(1)).toString(),
+        is("[(0‥1), (1‥+∞)]"));
+    assertThat(RangeSets.minus(setComplex, Range.singleton(1)).toString(),
+        is("[[0‥1), (1‥2], [3‥3], (5‥+∞)]"));
+    assertThat(RangeSets.minus(setComplex, Range.singleton(2)).toString(),
+        is("[[0‥2), [3‥3], (5‥+∞)]"));
+    assertThat(RangeSets.minus(setComplex, Range.singleton(3)).toString(),
+        is("[[0‥2], (5‥+∞)]"));
+    assertThat(RangeSets.minus(setComplex, Range.open(2, 3)).toString(),
+        is("[[0‥2], [3‥3], (5‥+∞)]"));
+    assertThat(RangeSets.minus(setComplex, Range.closed(2, 3)).toString(),
+        is("[[0‥2), (5‥+∞)]"));
+    assertThat(RangeSets.minus(setComplex, Range.closed(2, 7)).toString(),
+        is("[[0‥2), (7‥+∞)]"));
+  }
+
   @Test void testSimplifyOrTerms() {
     final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
     final RelDataType rowType = typeFactory.builder()
@@ -1459,17 +1501,47 @@ class RexProgramTest extends RexProgramTestBase {
             isNull(bRef)),
         "true");
 
-    // "b is not null or c is null" unchanged
+    // "b is null b > 1 or b <= 1" ==> "true"
+    checkSimplifyFilter(
+        or(isNull(bRef),
+            gt(bRef, literal(1)),
+            le(bRef, literal(1))),
+        "true");
+
+    // "b > 1 or b <= 1 or b is null" ==> "true"
+    checkSimplifyFilter(
+        or(gt(bRef, literal(1)),
+            le(bRef, literal(1)),
+            isNull(bRef)),
+        "true");
+
+    // "b <= 1 or b > 1 or b is null" ==> "true"
+    checkSimplifyFilter(
+        or(le(bRef, literal(1)),
+            gt(bRef, literal(1)),
+            isNull(bRef)),
+        "true");
+
+    // "b < 2 or b > 0 or b is null" ==> "true"
+    checkSimplifyFilter(
+        or(lt(bRef, literal(2)),
+            gt(bRef, literal(0)),
+            isNull(bRef)),
+        "true");
+
+    // "b is not null or c is null" unchanged,
+    // but "c is null" is moved to front
     checkSimplifyFilter(
         or(isNotNull(bRef),
             isNull(cRef)),
-        "OR(IS NOT NULL(?0.b), IS NULL(?0.c))");
+        "OR(IS NULL(?0.c), IS NOT NULL(?0.b))");
 
-    // "b is null or b is not false" unchanged
+    // "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)
     checkSimplifyFilter(
         or(isNull(bRef),
             isNotFalse(bRef)),
-        "OR(IS NULL(?0.b), IS NOT FALSE(?0.b))");
+        "OR(IS NULL(?0.b), ?0.b)");
 
     // multiple predicates are handled correctly
     checkSimplifyFilter(
@@ -2659,17 +2731,28 @@ class RexProgramTest extends RexProgramTestBase {
   }
 
   @Test void testSimplifyNotEqual() {
-    RexNode ref = input(tInt(), 0);
+    final RexNode ref = input(tInt(), 0);
     RelOptPredicateList relOptPredicateList = RelOptPredicateList.of(rexBuilder,
         ImmutableList.of(eq(ref, literal(9))));
     checkSimplifyFilter(ne(ref, literal(9)), relOptPredicateList, "false");
     checkSimplifyFilter(ne(ref, literal(5)), relOptPredicateList, "true");
 
-    ref = input(tInt(true), 0);
-    checkSimplifyFilter(ne(ref, literal(9)), relOptPredicateList,
-        "AND(null, IS NULL($0))");
-    checkSimplifyFilter(ne(ref, literal(5)), relOptPredicateList,
-        "OR(null, IS NOT NULL($0))");
+    final RexNode refNullable = input(tInt(true), 0);
+    checkSimplifyFilter(ne(refNullable, literal(9)), relOptPredicateList,
+        "false");
+    checkSimplifyFilter(ne(refNullable, literal(5)), relOptPredicateList,
+        "IS NOT NULL($0)");
+  }
+
+  /** Tests
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-4094">[CALCITE-4094]
+   * 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");
+    checkSimplifyUnchanged(like(ref, literal("%A")));
+    checkSimplifyUnchanged(like(ref, literal("%A"), literal("#")));
   }
 
   @Test void testSimplifyNonDeterministicFunction() {
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index 0d88ff7..c5aa7c0 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -95,6 +95,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.BiConsumer;
 import java.util.function.Function;
 import java.util.function.ObjIntConsumer;
+import java.util.function.Predicate;
 import java.util.function.UnaryOperator;
 
 import static org.apache.calcite.test.Matchers.isLinux;
@@ -2566,6 +2567,28 @@ class UtilTest {
         isIterable(Arrays.asList("John", "Paul", "Ringo")));
   }
 
+  /** Tests {@link Util#moveToHead(List, Predicate)}. */
+  @Test void testMoveToHead() {
+    final List<Integer> primes = ImmutableList.of(2, 3, 5, 7);
+    final List<Integer> evenInMiddle = ImmutableList.of(1, 2, 3);
+    final List<Integer> evenAtEnd = ImmutableList.of(1, 3, 8);
+    final List<Integer> empty = ImmutableList.of();
+    final List<Integer> evens = ImmutableList.of(0, 2, 4);
+    final List<Integer> odds = ImmutableList.of(1, 3, 5);
+    final Predicate<Integer> isEven = i -> i % 2 == 0;
+    assertThat(Util.moveToHead(primes, isEven).toString(), is("[2, 3, 5, 7]"));
+    assertThat(Util.moveToHead(primes, isEven), sameInstance(primes));
+    assertThat(Util.moveToHead(evenInMiddle, isEven).toString(),
+        is("[2, 1, 3]"));
+    assertThat(Util.moveToHead(evenAtEnd, isEven).toString(), is("[8, 1, 3]"));
+    assertThat(Util.moveToHead(empty, isEven).toString(), is("[]"));
+    assertThat(Util.moveToHead(empty, isEven), sameInstance(empty));
+    assertThat(Util.moveToHead(evens, isEven).toString(), is("[0, 2, 4]"));
+    assertThat(Util.moveToHead(evens, isEven), sameInstance(evens));
+    assertThat(Util.moveToHead(odds, isEven).toString(), is("[1, 3, 5]"));
+    assertThat(Util.moveToHead(odds, isEven), sameInstance(odds));
+  }
+
   /** Tests {@link Util#select(List, List)}. */
   @Test void testSelect() {
     final List<String> beatles =


[calcite] 01/03: Refactor

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 dc7bcc550caa5232b51200640c208769fb22787c
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Mon Aug 10 18:40:15 2020 -0700

    Refactor
    
    In RexProgramTest, construct literals when needed, rather than caching in variables.
    
    The word 'comparison' is mis-spelled in a few places.
    
    Deprecate RelReferentialConstraint.getNumColumns().
    
    Change TableFunctionScanFactory argument from RexNode to RexCall.
---
 .../calcite/rel/RelReferentialConstraint.java      |  11 +-
 .../calcite/rel/RelReferentialConstraintImpl.java  |   4 -
 .../org/apache/calcite/rel/core/RelFactories.java  |   7 +-
 .../rules/materialize/MaterializedViewRule.java    |  28 +-
 .../main/java/org/apache/calcite/sql/SqlKind.java  |   2 +-
 .../sql/validate/implicit/TypeCoercionImpl.java    |   6 +-
 .../java/org/apache/calcite/tools/RelBuilder.java  |   6 +-
 .../apache/calcite/rex/RexProgramBuilderBase.java  |  21 +-
 .../org/apache/calcite/rex/RexProgramTest.java     | 354 +++++++++------------
 9 files changed, 201 insertions(+), 238 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraint.java b/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraint.java
index 504f20f..6dc3c14 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraint.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraint.java
@@ -27,10 +27,13 @@ import java.util.List;
 public interface RelReferentialConstraint {
   //~ Methods ----------------------------------------------------------------
 
-  /**
-   * Returns the number of columns in the keys.
-   */
-  int getNumColumns();
+  /** Returns the number of columns in the keys.
+   *
+   * @deprecated Use {@code getColumnPairs().size()} */
+  @Deprecated // to be removed before 2.0
+  default int getNumColumns() {
+    return getColumnPairs().size();
+  }
 
   /**The qualified name of the referencing table, e.g. DEPT. */
   List<String> getSourceQualifiedName();
diff --git a/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraintImpl.java b/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraintImpl.java
index 0a62d12..ea25617 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraintImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraintImpl.java
@@ -48,10 +48,6 @@ public class RelReferentialConstraintImpl implements RelReferentialConstraint {
     return columnPairs;
   }
 
-  @Override public int getNumColumns() {
-    return columnPairs.size();
-  }
-
   public static RelReferentialConstraintImpl of(List<String> sourceQualifiedName,
       List<String> targetQualifiedName, List<IntPair> columnPairs) {
     return new RelReferentialConstraintImpl(
diff --git a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
index 2649dce..ce0a00b 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
@@ -503,7 +503,7 @@ public class RelFactories {
   public interface TableFunctionScanFactory {
     /** Creates a {@link TableFunctionScan}. */
     RelNode createTableFunctionScan(RelOptCluster cluster,
-        List<RelNode> inputs, RexNode rexCall, Type elementType,
+        List<RelNode> inputs, RexCall call, Type elementType,
         Set<RelColumnMapping> columnMappings);
   }
 
@@ -515,9 +515,8 @@ public class RelFactories {
   private static class TableFunctionScanFactoryImpl
       implements TableFunctionScanFactory {
     @Override public RelNode createTableFunctionScan(RelOptCluster cluster,
-        List<RelNode> inputs, RexNode rexCall, Type elementType,
+        List<RelNode> inputs, RexCall call, Type elementType,
         Set<RelColumnMapping> columnMappings) {
-      final RexCall call = (RexCall) rexCall;
       final SqlOperatorBinding callBinding =
           new RexCallBinding(cluster.getTypeFactory(), call.getOperator(),
               call.operands, ImmutableList.of());
@@ -525,7 +524,7 @@ public class RelFactories {
       final SqlReturnTypeInference rowTypeInference =
           operator.getRowTypeInference();
       final RelDataType rowType = rowTypeInference.inferReturnType(callBinding);
-      return LogicalTableFunctionScan.create(cluster, inputs, rexCall,
+      return LogicalTableFunctionScan.create(cluster, inputs, call,
           elementType, rowType, columnMappings);
     }
   }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/materialize/MaterializedViewRule.java b/core/src/main/java/org/apache/calcite/rel/rules/materialize/MaterializedViewRule.java
index 8ff7e43..cabd894 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/materialize/MaterializedViewRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/materialize/MaterializedViewRule.java
@@ -34,6 +34,7 @@ import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexExecutor;
@@ -53,6 +54,7 @@ import org.apache.calcite.util.Util;
 import org.apache.calcite.util.graph.DefaultDirectedGraph;
 import org.apache.calcite.util.graph.DefaultEdge;
 import org.apache.calcite.util.graph.DirectedGraph;
+import org.apache.calcite.util.mapping.IntPair;
 import org.apache.calcite.util.mapping.Mapping;
 
 import com.google.common.collect.ArrayListMultimap;
@@ -719,17 +721,21 @@ public abstract class MaterializedViewRule<C extends MaterializedViewRule.Config
             tableVNameToTableRefs.get(constraint.getTargetQualifiedName());
         for (RelTableRef parentTRef : parentTableRefs) {
           boolean canBeRewritten = true;
-          Multimap<RexTableInputRef, RexTableInputRef> equiColumns =
-                  ArrayListMultimap.create();
-          for (int pos = 0; pos < constraint.getNumColumns(); pos++) {
-            int foreignKeyPos = constraint.getColumnPairs().get(pos).source;
-            RelDataType foreignKeyColumnType =
-                tRef.getTable().getRowType().getFieldList().get(foreignKeyPos).getType();
-            RexTableInputRef foreignKeyColumnRef =
-                RexTableInputRef.of(tRef, foreignKeyPos, foreignKeyColumnType);
-            int uniqueKeyPos = constraint.getColumnPairs().get(pos).target;
-            RexTableInputRef uniqueKeyColumnRef = RexTableInputRef.of(parentTRef, uniqueKeyPos,
-                parentTRef.getTable().getRowType().getFieldList().get(uniqueKeyPos).getType());
+          final Multimap<RexTableInputRef, RexTableInputRef> equiColumns =
+              ArrayListMultimap.create();
+          final List<RelDataTypeField> foreignFields =
+              tRef.getTable().getRowType().getFieldList();
+          final List<RelDataTypeField> uniqueFields =
+              parentTRef.getTable().getRowType().getFieldList();
+          for (IntPair pair : constraint.getColumnPairs()) {
+            final RelDataType foreignKeyColumnType =
+                foreignFields.get(pair.source).getType();
+            final RexTableInputRef foreignKeyColumnRef =
+                RexTableInputRef.of(tRef, pair.source, foreignKeyColumnType);
+            final RelDataType uniqueKeyColumnType =
+                uniqueFields.get(pair.target).getType();
+            final RexTableInputRef uniqueKeyColumnRef =
+                RexTableInputRef.of(parentTRef, pair.target, uniqueKeyColumnType);
             if (!foreignKeyColumnType.isNullable()
                 && sourceEC.getEquivalenceClassesMap().containsKey(uniqueKeyColumnRef)
                 && sourceEC.getEquivalenceClassesMap().get(uniqueKeyColumnRef)
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 caaa6ba..8921ce3 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -1200,7 +1200,7 @@ public enum SqlKind {
   /**
    * Simple binary operators are those operators which expects operands from the same Domain.
    *
-   * <p>Example: simple comparisions ({@code =}, {@code <}).
+   * <p>Example: simple comparisons ({@code =}, {@code <}).
    *
    * <p>Note: it does not contain {@code IN} because that is defined on D x D^n.
    */
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/implicit/TypeCoercionImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/implicit/TypeCoercionImpl.java
index 312683a..4018b22 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/implicit/TypeCoercionImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/implicit/TypeCoercionImpl.java
@@ -195,8 +195,8 @@ public class TypeCoercionImpl extends AbstractTypeCoercion {
    *   <li>For EQUALS(=) operator: 1. If operands are BOOLEAN and NUMERIC, evaluate
    *   `1=true` and `0=false` all to be true; 2. If operands are datetime and string,
    *   do nothing because the SqlToRelConverter already makes the type coercion;</li>
-   *   <li>For binary comparision [=, &gt;, &gt;=, &lt;, &lt;=]: try to find the common type,
-   *   i.e. "1 &gt; '1'" will be converted to "1 &gt; 1";</li>
+   *   <li>For binary comparison [=, &gt;, &gt;=, &lt;, &lt;=]: try to find the
+   *   common type, i.e. "1 &gt; '1'" will be converted to "1 &gt; 1";</li>
    *   <li>For BETWEEN operator, find the common comparison data type of all the operands,
    *   the common type is deduced from left to right, i.e. for expression "A between B and C",
    *   finds common comparison type D between A and B
@@ -220,7 +220,7 @@ public class TypeCoercionImpl extends AbstractTypeCoercion {
         // BOOLEAN and literal
         coerced = booleanEquality(binding, type1, type2) || coerced;
       }
-      // Binary comparision operator like: = > >= < <=
+      // Binary comparison operator like: = > >= < <=
       if (kind.belongsTo(SqlKind.BINARY_COMPARISON)) {
         final RelDataType commonType = commonTypeForBinaryComparison(type1, type2);
         if (null != commonType) {
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 9644253..2cb4079 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -581,10 +581,10 @@ public class RelBuilder {
   }
 
   /** Creates a call to a scalar operator. */
-  private @Nonnull RexNode call(SqlOperator operator, List<RexNode> operandList) {
+  private @Nonnull RexCall call(SqlOperator operator, List<RexNode> operandList) {
     final RexBuilder builder = cluster.getRexBuilder();
     final RelDataType type = builder.deriveReturnType(operator, operandList);
-    return builder.makeCall(type, operator, operandList);
+    return (RexCall) builder.makeCall(type, operator, operandList);
   }
 
   /** Creates a call to a scalar operator. */
@@ -1148,7 +1148,7 @@ public class RelBuilder {
       inputs.add(0, build());
     }
 
-    final RexNode call = call(operator, ImmutableList.copyOf(operands));
+    final RexCall call = call(operator, ImmutableList.copyOf(operands));
     final RelNode functionScan =
         struct.tableFunctionScanFactory.createTableFunctionScan(cluster,
             inputs, call, null, getColumnMappings(operator));
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 1148414..0b60363 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramBuilderBase.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramBuilderBase.java
@@ -463,41 +463,42 @@ public abstract class RexProgramBuilderBase {
     return rexBuilder.makeNullLiteral(nullable(type));
   }
 
-  protected RexNode literal(boolean value) {
-    return rexBuilder.makeLiteral(value, nonNullableBool, false);
+  protected RexLiteral literal(boolean value) {
+    return (RexLiteral) rexBuilder.makeLiteral(value, nonNullableBool, false);
   }
 
-  protected RexNode literal(Boolean value) {
+  protected RexLiteral literal(Boolean value) {
     if (value == null) {
       return rexBuilder.makeNullLiteral(nullableBool);
     }
     return literal(value.booleanValue());
   }
 
-  protected RexNode literal(int value) {
-    return rexBuilder.makeLiteral(value, nonNullableInt, false);
+  protected RexLiteral literal(int value) {
+    return (RexLiteral) rexBuilder.makeLiteral(value, nonNullableInt, false);
   }
 
-  protected RexNode literal(BigDecimal value) {
+  protected RexLiteral literal(BigDecimal value) {
     return rexBuilder.makeExactLiteral(value);
   }
 
-  protected RexNode literal(BigDecimal value, RelDataType type) {
+  protected RexLiteral literal(BigDecimal value, RelDataType type) {
     return rexBuilder.makeExactLiteral(value, type);
   }
 
-  protected RexNode literal(Integer value) {
+  protected RexLiteral literal(Integer value) {
     if (value == null) {
       return rexBuilder.makeNullLiteral(nullableInt);
     }
     return literal(value.intValue());
   }
 
-  protected RexNode literal(String value) {
+  protected RexLiteral literal(String value) {
     if (value == null) {
       return rexBuilder.makeNullLiteral(nullableVarchar);
     }
-    return rexBuilder.makeLiteral(value, nonNullableVarchar, false);
+    return (RexLiteral) rexBuilder.makeLiteral(value, nonNullableVarchar,
+        false);
   }
 
   // Variables
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 5307ac0..8582007 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
@@ -224,21 +224,14 @@ class RexProgramTest extends RexProgramTestBase {
     // $t2 = $t0 + 1 (i.e. x + 1)
     final RexNode i0 = rexBuilder.makeInputRef(
         types.get(0), 0);
-    final RexLiteral c1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
-    final RexLiteral c5 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(5L));
     RexLocalRef t2 =
         builder.addExpr(
             rexBuilder.makeCall(
                 SqlStdOperatorTable.PLUS,
-                i0,
-                c1));
+                i0, literal(1)));
     // $t3 = 77 (not used)
-    final RexLiteral c77 =
-        rexBuilder.makeExactLiteral(
-            BigDecimal.valueOf(77));
     RexLocalRef t3 =
-        builder.addExpr(
-            c77);
+        builder.addExpr(literal(77));
     Util.discard(t3);
     // $t4 = $t0 + $t1 (i.e. x + y)
     final RexNode i1 = rexBuilder.makeInputRef(
@@ -270,8 +263,7 @@ class RexProgramTest extends RexProgramTestBase {
           builder.addExpr(
               rexBuilder.makeCall(
                   SqlStdOperatorTable.PLUS,
-                  i0,
-                  c1));
+                  i0, literal(1)));
       // $t5 = $t0 + $tx (i.e. x + (x + 1))
       t5 =
           builder.addExpr(
@@ -314,7 +306,7 @@ class RexProgramTest extends RexProgramTestBase {
     case 3:
     case 4:
       // $t7 = 5
-      t7 = builder.addExpr(c5);
+      t7 = builder.addExpr(literal(5));
       // $t8 = $t2 > $t7 (i.e. (x + 1) > 5)
       t8 = builder.addExpr(gt(t2, t7));
       // $t9 = true
@@ -555,9 +547,7 @@ class RexProgramTest extends RexProgramTestBase {
     final RexNode gRef = rexBuilder.makeFieldAccess(range, 6);
     final RexNode hRef = rexBuilder.makeFieldAccess(range, 7);
 
-    final RexLiteral sevenLiteral =
-        rexBuilder.makeExactLiteral(BigDecimal.valueOf(7));
-    final RexNode hEqSeven = eq(hRef, sevenLiteral);
+    final RexNode hEqSeven = eq(hRef, literal(7));
 
     checkCnf(aRef, "?0.a");
     checkCnf(trueLiteral, "true");
@@ -624,24 +614,17 @@ class RexProgramTest extends RexProgramTestBase {
     final RexNode aRef = rexBuilder.makeFieldAccess(range, 3);
     final RexNode bRef = rexBuilder.makeFieldAccess(range, 4);
 
-    final RexLiteral literal1 =
-        rexBuilder.makeExactLiteral(BigDecimal.valueOf(1));
-    final RexLiteral literal2 =
-        rexBuilder.makeExactLiteral(BigDecimal.valueOf(2));
-    final RexLiteral literal3 =
-        rexBuilder.makeExactLiteral(BigDecimal.valueOf(3));
-
     checkCnf(
         or(
-            and(eq(xRef, literal1),
-                eq(yRef, literal1),
-                eq(zRef, literal1)),
-            and(eq(xRef, literal2),
-                eq(yRef, literal2),
-                eq(aRef, literal2)),
-            and(eq(xRef, literal3),
-                eq(aRef, literal3),
-                eq(bRef, literal3))),
+            and(eq(xRef, literal(1)),
+                eq(yRef, literal(1)),
+                eq(zRef, literal(1))),
+            and(eq(xRef, literal(2)),
+                eq(yRef, literal(2)),
+                eq(aRef, literal(2))),
+            and(eq(xRef, literal(3)),
+                eq(aRef, literal(3)),
+                eq(bRef, literal(3)))),
         "AND("
             + "OR(=(?0.x, 1), =(?0.x, 2), =(?0.x, 3)), "
             + "OR(=(?0.x, 1), =(?0.x, 2), =(?0.a, 3)), "
@@ -686,22 +669,14 @@ class RexProgramTest extends RexProgramTestBase {
     final RexNode xRef = rexBuilder.makeFieldAccess(range, 0);
     final RexNode yRef = rexBuilder.makeFieldAccess(range, 1);
 
-    final RexLiteral literal1 =
-        rexBuilder.makeExactLiteral(BigDecimal.valueOf(1));
-    final RexLiteral literal2 =
-        rexBuilder.makeExactLiteral(BigDecimal.valueOf(2));
-    final RexLiteral literal3 =
-        rexBuilder.makeExactLiteral(BigDecimal.valueOf(3));
-    final RexLiteral literal4 =
-        rexBuilder.makeExactLiteral(BigDecimal.valueOf(4));
-
     // Expression
     //   OR(=(?0.x, 1), AND(=(?0.x, 2), =(?0.y, 3)))
     // transformation creates 7 nodes
     //   AND(OR(=(?0.x, 1), =(?0.x, 2)), OR(=(?0.x, 1), =(?0.y, 3)))
     // Thus, it is triggered.
     checkThresholdCnf(
-        or(eq(xRef, literal1), and(eq(xRef, literal2), eq(yRef, literal3))),
+        or(eq(xRef, literal(1)),
+            and(eq(xRef, literal(2)), eq(yRef, literal(3)))),
         8, "AND(OR(=(?0.x, 1), =(?0.x, 2)), OR(=(?0.x, 1), =(?0.y, 3)))");
 
     // Expression
@@ -711,8 +686,8 @@ class RexProgramTest extends RexProgramTestBase {
     //       OR(=(?0.x, 1), =(?0.x, 2), =(?0.y, 8)))
     // Thus, it is NOT triggered.
     checkThresholdCnf(
-        or(eq(xRef, literal1), eq(xRef, literal2),
-            and(eq(xRef, literal3), eq(yRef, literal4))),
+        or(eq(xRef, literal(1)), eq(xRef, literal(2)),
+            and(eq(xRef, literal(3)), eq(yRef, literal(4)))),
                 8, "OR(=(?0.x, 1), =(?0.x, 2), AND(=(?0.x, 3), =(?0.y, 4)))");
   }
 
@@ -780,9 +755,7 @@ class RexProgramTest extends RexProgramTestBase {
     final RexNode gRef = rexBuilder.makeFieldAccess(range, 6);
     final RexNode hRef = rexBuilder.makeFieldAccess(range, 7);
 
-    final RexLiteral sevenLiteral =
-        rexBuilder.makeExactLiteral(BigDecimal.valueOf(7));
-    final RexNode hEqSeven = eq(hRef, sevenLiteral);
+    final RexNode hEqSeven = eq(hRef, literal(7));
 
     // Most of the expressions in testCnf are unaffected by pullFactors.
     checkPullFactors(
@@ -855,7 +828,6 @@ class RexProgramTest extends RexProgramTestBase {
     final RexNode iRef = rexBuilder.makeFieldAccess(range, 8);
     final RexNode jRef = rexBuilder.makeFieldAccess(range, 9);
     final RexNode kRef = rexBuilder.makeFieldAccess(range, 10);
-    final RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
 
     // and: remove duplicates
     checkSimplify(and(aRef, bRef, aRef), "AND(?0.a, ?0.b)");
@@ -912,7 +884,8 @@ class RexProgramTest extends RexProgramTestBase {
 
     // case: always same value
     checkSimplify(
-        case_(aRef, literal1, bRef, literal1, cRef, literal1, dRef, literal1, literal1), "1");
+        case_(aRef, literal(1), bRef, literal(1), cRef, literal(1), dRef,
+            literal(1), literal(1)), "1");
 
     // case: trailing false and null, no simplification
     checkSimplify3(
@@ -951,69 +924,69 @@ class RexProgramTest extends RexProgramTestBase {
         "true");
 
     // condition, and the inverse - nothing to do due to null values
-    checkSimplify2(and(le(hRef, literal1), gt(hRef, literal1)),
+    checkSimplify2(and(le(hRef, literal(1)), gt(hRef, literal(1))),
         "AND(<=(?0.h, 1), >(?0.h, 1))",
         "false");
 
-    checkSimplify2(and(le(hRef, literal1), ge(hRef, literal1)),
+    checkSimplify2(and(le(hRef, literal(1)), ge(hRef, literal(1))),
         "AND(<=(?0.h, 1), >=(?0.h, 1))",
         "=(?0.h, 1)");
 
-    checkSimplify2(and(lt(hRef, literal1), eq(hRef, literal1), ge(hRef, literal1)),
+    checkSimplify2(and(lt(hRef, literal(1)), eq(hRef, literal(1)), ge(hRef, literal(1))),
         "AND(<(?0.h, 1), =(?0.h, 1), >=(?0.h, 1))",
         "false");
 
-    checkSimplify(and(lt(hRef, literal1), or(falseLiteral, falseLiteral)),
+    checkSimplify(and(lt(hRef, literal(1)), or(falseLiteral, falseLiteral)),
         "false");
-    checkSimplify(and(lt(hRef, literal1), or(falseLiteral, gt(jRef, kRef))),
+    checkSimplify(and(lt(hRef, literal(1)), or(falseLiteral, gt(jRef, kRef))),
         "AND(<(?0.h, 1), >(?0.j, ?0.k))");
-    checkSimplify(or(lt(hRef, literal1), and(trueLiteral, trueLiteral)),
+    checkSimplify(or(lt(hRef, literal(1)), and(trueLiteral, trueLiteral)),
         "true");
     checkSimplify(
-        or(lt(hRef, literal1),
+        or(lt(hRef, literal(1)),
             and(trueLiteral, or(trueLiteral, falseLiteral))),
         "true");
     checkSimplify(
-        or(lt(hRef, literal1),
+        or(lt(hRef, literal(1)),
             and(trueLiteral, and(trueLiteral, falseLiteral))),
         "<(?0.h, 1)");
     checkSimplify(
-        or(lt(hRef, literal1),
+        or(lt(hRef, literal(1)),
             and(trueLiteral, or(falseLiteral, falseLiteral))),
         "<(?0.h, 1)");
 
     // "x = x" simplifies to "x is not null"
-    checkSimplify(eq(literal1, literal1), "true");
+    checkSimplify(eq(literal(1), literal(1)), "true");
     checkSimplify(eq(hRef, hRef), "true");
     checkSimplify3(eq(iRef, iRef), "OR(null, IS NOT NULL(?0.i))", "IS NOT NULL(?0.i)", "true");
     checkSimplifyUnchanged(eq(iRef, hRef));
 
     // "x <= x" simplifies to "x is not null"
-    checkSimplify(le(literal1, literal1), "true");
+    checkSimplify(le(literal(1), literal(1)), "true");
     checkSimplify(le(hRef, hRef), "true");
     checkSimplify3(le(iRef, iRef), "OR(null, IS NOT NULL(?0.i))", "IS NOT NULL(?0.i)", "true");
     checkSimplifyUnchanged(le(iRef, hRef));
 
     // "x >= x" simplifies to "x is not null"
-    checkSimplify(ge(literal1, literal1), "true");
+    checkSimplify(ge(literal(1), literal(1)), "true");
     checkSimplify(ge(hRef, hRef), "true");
     checkSimplify3(ge(iRef, iRef), "OR(null, IS NOT NULL(?0.i))", "IS NOT NULL(?0.i)", "true");
     checkSimplifyUnchanged(ge(iRef, hRef));
 
-    // "x != x" simplifies to "false"
-    checkSimplify(ne(literal1, literal1), "false");
+    // "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)");
     checkSimplifyUnchanged(ne(iRef, hRef));
 
     // "x < x" simplifies to "false"
-    checkSimplify(lt(literal1, literal1), "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)");
     checkSimplifyUnchanged(lt(iRef, hRef));
 
     // "x > x" simplifies to "false"
-    checkSimplify(gt(literal1, literal1), "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)");
     checkSimplifyUnchanged(gt(iRef, hRef));
@@ -1085,24 +1058,22 @@ class RexProgramTest extends RexProgramTestBase {
     final RexNode dRef = rexBuilder.makeFieldAccess(range, 3);
     final RexNode eRef = rexBuilder.makeFieldAccess(range, 4);
     final RexNode fRef = rexBuilder.makeFieldAccess(range, 5);
-    final RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
-    final RexLiteral literal5 = rexBuilder.makeExactLiteral(new BigDecimal(5));
-    final RexLiteral literal10 = rexBuilder.makeExactLiteral(BigDecimal.TEN);
 
     // condition, and the inverse
-    checkSimplifyFilter(and(le(aRef, literal1), gt(aRef, literal1)),
+    checkSimplifyFilter(and(le(aRef, literal(1)), gt(aRef, literal(1))),
         "false");
 
-    checkSimplifyFilter(and(le(aRef, literal1), ge(aRef, literal1)),
+    checkSimplifyFilter(and(le(aRef, literal(1)), ge(aRef, literal(1))),
         "=(?0.a, 1)");
 
-    checkSimplifyFilter(and(lt(aRef, literal1), eq(aRef, literal1), ge(aRef, literal1)),
+    checkSimplifyFilter(
+        and(lt(aRef, literal(1)), eq(aRef, literal(1)), ge(aRef, literal(1))),
         "false");
 
     // simplify equals boolean
     final ImmutableList<RexNode> args =
-        ImmutableList.of(eq(eq(aRef, literal1), trueLiteral),
-            eq(bRef, literal1));
+        ImmutableList.of(eq(eq(aRef, literal(1)), trueLiteral),
+            eq(bRef, literal(1)));
     checkSimplifyFilter(and(args),
         "AND(=(?0.a, 1), =(?0.b, 1))");
 
@@ -1114,7 +1085,7 @@ class RexProgramTest extends RexProgramTestBase {
 
     // "a = 1 and a = 10" is always false
     final ImmutableList<RexNode> args2 =
-        ImmutableList.of(eq(aRef, literal1), eq(aRef, literal10));
+        ImmutableList.of(eq(aRef, literal(1)), eq(aRef, literal(10)));
     checkSimplifyFilter(and(args2), "false");
 
     assertThat(simplify
@@ -1122,20 +1093,20 @@ class RexProgramTest extends RexProgramTestBase {
         nullValue());
 
     // equality on constants, can remove the equality on the variables
-    checkSimplifyFilter(and(eq(aRef, literal1), eq(bRef, literal1), eq(aRef, bRef)),
+    checkSimplifyFilter(and(eq(aRef, literal(1)), eq(bRef, literal(1)), eq(aRef, bRef)),
         "AND(=(?0.a, 1), =(?0.b, 1))");
 
     // condition not satisfiable
-    checkSimplifyFilter(and(eq(aRef, literal1), eq(bRef, literal10), eq(aRef, bRef)),
+    checkSimplifyFilter(and(eq(aRef, literal(1)), eq(bRef, literal(10)), eq(aRef, bRef)),
         "false");
 
     // condition not satisfiable
-    checkSimplifyFilter(and(gt(aRef, literal10), ge(bRef, literal1), lt(aRef, literal10)),
+    checkSimplifyFilter(and(gt(aRef, literal(10)), ge(bRef, literal(1)), lt(aRef, literal(10))),
         "false");
 
     // one "and" containing three "or"s
     checkSimplifyFilter(
-        or(gt(aRef, literal10), gt(bRef, literal1), gt(aRef, literal10)),
+        or(gt(aRef, literal(10)), gt(bRef, literal(1)), gt(aRef, literal(10))),
         "OR(>(?0.a, 10), >(?0.b, 1))");
 
     // case: trailing false and null, remove
@@ -1145,36 +1116,36 @@ class RexProgramTest extends RexProgramTestBase {
         "OR(?0.c, ?0.d)");
 
     // condition with null value for range
-    checkSimplifyFilter(and(gt(aRef, nullBool), ge(bRef, literal1)), "false");
+    checkSimplifyFilter(and(gt(aRef, nullBool), ge(bRef, literal(1))), "false");
 
     // condition "1 < a && 5 < x" yields "5 < x"
     checkSimplifyFilter(
-        and(lt(literal1, aRef), lt(literal5, aRef)),
+        and(lt(literal(1), aRef), lt(literal(5), aRef)),
         RelOptPredicateList.EMPTY,
         "<(5, ?0.a)");
 
     // condition "1 < a && a < 5" is unchanged
     checkSimplifyFilter(
-        and(lt(literal1, aRef), lt(aRef, literal5)),
+        and(lt(literal(1), aRef), lt(aRef, literal(5))),
         RelOptPredicateList.EMPTY,
         "AND(<(1, ?0.a), <(?0.a, 5))");
 
     // condition "1 > a && 5 > x" yields "1 > a"
     checkSimplifyFilter(
-        and(gt(literal1, aRef), gt(literal5, aRef)),
+        and(gt(literal(1), aRef), gt(literal(5), aRef)),
         RelOptPredicateList.EMPTY,
         ">(1, ?0.a)");
 
     // condition "1 > a && a > 5" yields false
     checkSimplifyFilter(
-        and(gt(literal1, aRef), gt(aRef, literal5)),
+        and(gt(literal(1), aRef), gt(aRef, literal(5))),
         RelOptPredicateList.EMPTY,
         "false");
 
     // range with no predicates;
     // condition "a > 1 && a < 10 && a < 5" yields "a < 1 && a < 5"
     checkSimplifyFilter(
-        and(gt(aRef, literal1), lt(aRef, literal10), lt(aRef, literal5)),
+        and(gt(aRef, literal(1)), lt(aRef, literal(10)), lt(aRef, literal(5))),
         RelOptPredicateList.EMPTY,
         "AND(>(?0.a, 1), <(?0.a, 5))");
 
@@ -1182,9 +1153,9 @@ class RexProgramTest extends RexProgramTestBase {
     // with pre-condition "a > 5"
     // yields "false"
     checkSimplifyFilter(
-        and(gt(aRef, literal1), lt(aRef, literal10), lt(aRef, literal5)),
+        and(gt(aRef, literal(1)), lt(aRef, literal(10)), lt(aRef, literal(5))),
         RelOptPredicateList.of(rexBuilder,
-            ImmutableList.of(gt(aRef, literal5))),
+            ImmutableList.of(gt(aRef, literal(5)))),
         "false");
 
     // condition "a > 1 && a < 10 && a <= 5"
@@ -1192,66 +1163,66 @@ class RexProgramTest extends RexProgramTestBase {
     // yields "a = 5"
     // "a <= 5" would also be correct, just a little less concise.
     checkSimplifyFilter(
-        and(gt(aRef, literal1), lt(aRef, literal10), le(aRef, literal5)),
+        and(gt(aRef, literal(1)), lt(aRef, literal(10)), le(aRef, literal(5))),
         RelOptPredicateList.of(rexBuilder,
-            ImmutableList.of(ge(aRef, literal5))),
+            ImmutableList.of(ge(aRef, literal(5)))),
         "=(?0.a, 5)");
 
     // condition "a > 1 && a < 10 && a < 5"
     // with pre-condition "b < 10 && a > 5"
     // yields "a > 1 and a < 5"
     checkSimplifyFilter(
-        and(gt(aRef, literal1), lt(aRef, literal10), lt(aRef, literal5)),
+        and(gt(aRef, literal(1)), lt(aRef, literal(10)), lt(aRef, literal(5))),
         RelOptPredicateList.of(rexBuilder,
-            ImmutableList.of(lt(bRef, literal10), ge(aRef, literal1))),
+            ImmutableList.of(lt(bRef, literal(10)), ge(aRef, literal(1)))),
         "AND(>(?0.a, 1), <(?0.a, 5))");
 
     // condition "a > 1"
     // with pre-condition "b < 10 && a > 5"
     // yields "true"
-    checkSimplifyFilter(gt(aRef, literal1),
+    checkSimplifyFilter(gt(aRef, literal(1)),
         RelOptPredicateList.of(rexBuilder,
-            ImmutableList.of(lt(bRef, literal10), gt(aRef, literal5))),
+            ImmutableList.of(lt(bRef, literal(10)), gt(aRef, literal(5)))),
         "true");
 
     // condition "a < 1"
     // with pre-condition "b < 10 && a > 5"
     // yields "false"
-    checkSimplifyFilter(lt(aRef, literal1),
+    checkSimplifyFilter(lt(aRef, literal(1)),
         RelOptPredicateList.of(rexBuilder,
-            ImmutableList.of(lt(bRef, literal10), gt(aRef, literal5))),
+            ImmutableList.of(lt(bRef, literal(10)), gt(aRef, literal(5)))),
         "false");
 
     // condition "a > 5"
     // with pre-condition "b < 10 && a >= 5"
     // yields "a > 5"
-    checkSimplifyFilter(gt(aRef, literal5),
+    checkSimplifyFilter(gt(aRef, literal(5)),
         RelOptPredicateList.of(rexBuilder,
-            ImmutableList.of(lt(bRef, literal10), ge(aRef, literal5))),
+            ImmutableList.of(lt(bRef, literal(10)), ge(aRef, literal(5)))),
         ">(?0.a, 5)");
 
     // condition "a > 5"
     // with pre-condition "a <= 5"
     // yields "false"
-    checkSimplifyFilter(gt(aRef, literal5),
+    checkSimplifyFilter(gt(aRef, literal(5)),
         RelOptPredicateList.of(rexBuilder,
-            ImmutableList.of(le(aRef, literal5))),
+            ImmutableList.of(le(aRef, literal(5)))),
         "false");
 
     // condition "a > 5"
     // with pre-condition "a <= 5 and b <= 5"
     // yields "false"
-    checkSimplifyFilter(gt(aRef, literal5),
+    checkSimplifyFilter(gt(aRef, literal(5)),
         RelOptPredicateList.of(rexBuilder,
-            ImmutableList.of(le(aRef, literal5), le(bRef, literal5))),
+            ImmutableList.of(le(aRef, literal(5)), le(bRef, literal(5)))),
         "false");
 
     // condition "a > 5 or b > 5"
     // with pre-condition "a <= 5 and b <= 5"
     // should yield "false" but yields "a = 5 or b = 5"
-    checkSimplifyFilter(or(gt(aRef, literal5), gt(bRef, literal5)),
+    checkSimplifyFilter(or(gt(aRef, literal(5)), gt(bRef, literal(5))),
         RelOptPredicateList.of(rexBuilder,
-            ImmutableList.of(le(aRef, literal5), le(bRef, literal5))),
+            ImmutableList.of(le(aRef, literal(5)), le(bRef, literal(5)))),
         "false");
   }
 
@@ -1313,46 +1284,43 @@ class RexProgramTest extends RexProgramTestBase {
     final RexDynamicParam range = rexBuilder.makeDynamicParam(rowType, 0);
     final RexNode aRef = rexBuilder.makeFieldAccess(range, 0);
     final RexNode bRef = rexBuilder.makeFieldAccess(range, 1);
-    final RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
-    final RexLiteral literal5 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(5));
-    final RexLiteral literal10 = rexBuilder.makeExactLiteral(BigDecimal.TEN);
 
     checkSimplifyFilter(
         or(
-            or(eq(aRef, literal1),
-                eq(aRef, literal1)),
-            eq(aRef, literal1)),
+            or(eq(aRef, literal(1)),
+                eq(aRef, literal(1))),
+            eq(aRef, literal(1))),
         "=(?0.a, 1)");
 
     checkSimplifyFilter(
         or(
-            and(eq(aRef, literal1),
-                eq(aRef, literal1)),
-            and(eq(aRef, literal10),
-                eq(aRef, literal1))),
+            and(eq(aRef, literal(1)),
+                eq(aRef, literal(1))),
+            and(eq(aRef, literal(10)),
+                eq(aRef, literal(1)))),
         "=(?0.a, 1)");
 
     checkSimplifyFilter(
         and(
-            eq(aRef, literal1),
-            or(eq(aRef, literal1),
-                eq(aRef, literal10))),
+            eq(aRef, literal(1)),
+            or(eq(aRef, literal(1)),
+                eq(aRef, literal(10)))),
         "=(?0.a, 1)");
     checkSimplifyFilter(
         and(
-            or(eq(aRef, literal1),
-                eq(aRef, literal10)),
-            eq(aRef, literal1)),
+            or(eq(aRef, literal(1)),
+                eq(aRef, literal(10))),
+            eq(aRef, literal(1))),
         "=(?0.a, 1)");
 
     checkSimplifyFilter(
-        and(gt(aRef, literal10),
-            gt(aRef, literal1)),
+        and(gt(aRef, literal(10)),
+            gt(aRef, literal(1))),
         ">(?0.a, 10)");
 
     checkSimplifyFilter(
-        and(gt(aRef, literal1),
-            gt(aRef, literal10)),
+        and(gt(aRef, literal(1)),
+            gt(aRef, literal(10))),
         ">(?0.a, 10)");
 
     // "null AND NOT(null OR x)" => "null AND NOT(x)"
@@ -1386,27 +1354,23 @@ class RexProgramTest extends RexProgramTestBase {
     final RexNode aRef = rexBuilder.makeFieldAccess(range, 0);
     final RexNode bRef = rexBuilder.makeFieldAccess(range, 1);
     final RexNode cRef = rexBuilder.makeFieldAccess(range, 2);
-    final RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
-    final RexLiteral literal2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(2));
-    final RexLiteral literal3 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(3));
-    final RexLiteral literal4 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(4));
 
     // "a != 1 or a = 1" ==> "true"
     checkSimplifyFilter(
-        or(ne(aRef, literal1),
-            eq(aRef, literal1)),
+        or(ne(aRef, literal(1)),
+            eq(aRef, literal(1))),
         "true");
 
     // TODO: make this simplify to "true"
     checkSimplifyFilter(
-        or(eq(aRef, literal1),
-            ne(aRef, literal1)),
+        or(eq(aRef, literal(1)),
+            ne(aRef, literal(1))),
         "OR(=(?0.a, 1), <>(?0.a, 1))");
 
     // "b != 1 or b = 1" cannot be simplified, because b might be null
     final RexNode neOrEq =
-        or(ne(bRef, literal1),
-            eq(bRef, literal1));
+        or(ne(bRef, literal(1)),
+            eq(bRef, literal(1)));
     checkSimplifyFilter(neOrEq, "OR(<>(?0.b, 1), =(?0.b, 1))");
 
     // Careful of the excluded middle!
@@ -1451,12 +1415,12 @@ class RexProgramTest extends RexProgramTestBase {
     // multiple predicates are handled correctly
     checkSimplifyFilter(
         and(
-            or(eq(bRef, literal1),
-                eq(bRef, literal2)),
-            eq(bRef, literal2),
-            eq(aRef, literal3),
-            or(eq(aRef, literal3),
-                eq(aRef, literal4))),
+            or(eq(bRef, literal(1)),
+                eq(bRef, literal(2))),
+            eq(bRef, literal(2)),
+            eq(aRef, literal(3)),
+            or(eq(aRef, literal(3)),
+                eq(aRef, literal(4)))),
         "AND(=(?0.b, 2), =(?0.a, 3))");
 
     checkSimplify3(
@@ -1500,11 +1464,9 @@ class RexProgramTest extends RexProgramTestBase {
 
     final RexDynamicParam range = rexBuilder.makeDynamicParam(rowType, 0);
     final RexNode aRef = rexBuilder.makeFieldAccess(range, 0);
-    final RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
-
 
     checkSimplify2(
-        and(eq(aRef, literal1),
+        and(eq(aRef, literal(1)),
             nullInt),
         "AND(=(?0.a, 1), null:INTEGER)",
         "false");
@@ -1521,13 +1483,13 @@ class RexProgramTest extends RexProgramTestBase {
 
     checkSimplify3(
         and(nullBool,
-            eq(aRef, literal1)),
+            eq(aRef, literal(1))),
         "AND(null, =(?0.a, 1))",
         "false",
         "=(?0.a, 1)");
 
     checkSimplify3(
-        or(eq(aRef, literal1),
+        or(eq(aRef, literal(1)),
             nullBool),
         "OR(=(?0.a, 1), null)",
         "=(?0.a, 1)",
@@ -1592,7 +1554,7 @@ class RexProgramTest extends RexProgramTestBase {
     assertThat("The case should be nonNullable", caseNode.getType().isNullable(), is(false));
     assertThat("Expected a nonNullable type", result.getType().isNullable(), is(false));
     assertThat(result.getType().getSqlTypeName(), is(SqlTypeName.BOOLEAN));
-    assertThat(result.getOperator(), is((SqlOperator) SqlStdOperatorTable.IS_TRUE));
+    assertThat(result.getOperator(), is(SqlStdOperatorTable.IS_TRUE));
     assertThat(result.getOperands().get(0), is(condition));
   }
 
@@ -1770,13 +1732,12 @@ class RexProgramTest extends RexProgramTestBase {
     final RexInputRef i1 = rexBuilder.makeInputRef(intNullableType, 1);
     final RexInputRef i2 = rexBuilder.makeInputRef(intType, 2);
     final RexInputRef i3 = rexBuilder.makeInputRef(intType, 3);
-    final RexLiteral one = rexBuilder.makeExactLiteral(BigDecimal.ONE);
     final RexLiteral null_ = rexBuilder.makeNullLiteral(intType);
     checkSimplify(isNotNull(lt(i0, i1)),
         "AND(IS NOT NULL($0), IS NOT NULL($1))");
     checkSimplify(isNotNull(lt(i0, i2)), "IS NOT NULL($0)");
     checkSimplify(isNotNull(lt(i2, i3)), "true");
-    checkSimplify(isNotNull(lt(i0, one)), "IS NOT NULL($0)");
+    checkSimplify(isNotNull(lt(i0, literal(1))), "IS NOT NULL($0)");
     checkSimplify(isNotNull(lt(i0, null_)), "false");
     // test simplify operand of case when expression
     checkSimplify(
@@ -1933,7 +1894,6 @@ class RexProgramTest extends RexProgramTestBase {
 
   @Test void testSimplifyCastLiteral2() {
     final RexLiteral literalAbc = rexBuilder.makeLiteral("abc");
-    final RexLiteral literalOne = rexBuilder.makeExactLiteral(BigDecimal.ONE);
     final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
     final RelDataType varcharType =
         typeFactory.createSqlType(SqlTypeName.VARCHAR, 10);
@@ -1943,17 +1903,17 @@ class RexProgramTest extends RexProgramTestBase {
     final RelDataType timestampType =
         typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
     checkSimplifyUnchanged(cast(literalAbc, intType));
-    checkSimplifyUnchanged(cast(literalOne, intType));
+    checkSimplifyUnchanged(cast(literal(1), intType));
     checkSimplifyUnchanged(cast(literalAbc, varcharType));
-    checkSimplify(cast(literalOne, varcharType), "'1':VARCHAR(10)");
+    checkSimplify(cast(literal(1), varcharType), "'1':VARCHAR(10)");
     checkSimplifyUnchanged(cast(literalAbc, booleanType));
-    checkSimplify(cast(literalOne, booleanType),
+    checkSimplify(cast(literal(1), booleanType),
         "false"); // different from Hive
     checkSimplifyUnchanged(cast(literalAbc, dateType));
-    checkSimplify(cast(literalOne, dateType),
+    checkSimplify(cast(literal(1), dateType),
         "1970-01-02"); // different from Hive
     checkSimplifyUnchanged(cast(literalAbc, timestampType));
-    checkSimplify(cast(literalOne, timestampType),
+    checkSimplify(cast(literal(1), timestampType),
         "1970-01-01 00:00:00"); // different from Hive
   }
 
@@ -2058,10 +2018,8 @@ class RexProgramTest extends RexProgramTestBase {
   @Test void testSimplifyLiterals() {
     final RexLiteral literalAbc = rexBuilder.makeLiteral("abc");
     final RexLiteral literalDef = rexBuilder.makeLiteral("def");
-
-    final RexLiteral literalZero = rexBuilder.makeExactLiteral(BigDecimal.ZERO);
-    final RexLiteral literalOne = rexBuilder.makeExactLiteral(BigDecimal.ONE);
-    final RexLiteral literalOneDotZero = rexBuilder.makeExactLiteral(new BigDecimal(1.0));
+    final RexLiteral literalOneDotZero =
+        rexBuilder.makeExactLiteral(new BigDecimal(1D));
 
     // Check string comparison
     checkSimplify(eq(literalAbc, literalAbc), "true");
@@ -2082,48 +2040,48 @@ class RexProgramTest extends RexProgramTestBase {
     checkSimplify(le(literalDef, literalDef), "true");
 
     // Check whole number comparison
-    checkSimplify(eq(literalZero, literalOne), "false");
-    checkSimplify(eq(literalOne, literalZero), "false");
-    checkSimplify(ne(literalZero, literalOne), "true");
-    checkSimplify(ne(literalOne, literalZero), "true");
-    checkSimplify(gt(literalZero, literalOne), "false");
-    checkSimplify(gt(literalOne, literalZero), "true");
-    checkSimplify(gt(literalOne, literalOne), "false");
-    checkSimplify(ge(literalZero, literalOne), "false");
-    checkSimplify(ge(literalOne, literalZero), "true");
-    checkSimplify(ge(literalOne, literalOne), "true");
-    checkSimplify(lt(literalZero, literalOne), "true");
-    checkSimplify(lt(literalOne, literalZero), "false");
-    checkSimplify(lt(literalOne, literalOne), "false");
-    checkSimplify(le(literalZero, literalOne), "true");
-    checkSimplify(le(literalOne, literalZero), "false");
-    checkSimplify(le(literalOne, literalOne), "true");
+    checkSimplify(eq(literal(0), literal(1)), "false");
+    checkSimplify(eq(literal(1), literal(0)), "false");
+    checkSimplify(ne(literal(0), literal(1)), "true");
+    checkSimplify(ne(literal(1), literal(0)), "true");
+    checkSimplify(gt(literal(0), literal(1)), "false");
+    checkSimplify(gt(literal(1), literal(0)), "true");
+    checkSimplify(gt(literal(1), literal(1)), "false");
+    checkSimplify(ge(literal(0), literal(1)), "false");
+    checkSimplify(ge(literal(1), literal(0)), "true");
+    checkSimplify(ge(literal(1), literal(1)), "true");
+    checkSimplify(lt(literal(0), literal(1)), "true");
+    checkSimplify(lt(literal(1), literal(0)), "false");
+    checkSimplify(lt(literal(1), literal(1)), "false");
+    checkSimplify(le(literal(0), literal(1)), "true");
+    checkSimplify(le(literal(1), literal(0)), "false");
+    checkSimplify(le(literal(1), literal(1)), "true");
 
     // Check decimal equality comparison
-    checkSimplify(eq(literalOne, literalOneDotZero), "true");
-    checkSimplify(eq(literalOneDotZero, literalOne), "true");
-    checkSimplify(ne(literalOne, literalOneDotZero), "false");
-    checkSimplify(ne(literalOneDotZero, literalOne), "false");
+    checkSimplify(eq(literal(1), literalOneDotZero), "true");
+    checkSimplify(eq(literalOneDotZero, literal(1)), "true");
+    checkSimplify(ne(literal(1), literalOneDotZero), "false");
+    checkSimplify(ne(literalOneDotZero, literal(1)), "false");
 
     // Check different types shouldn't change simplification
-    checkSimplifyUnchanged(eq(literalZero, literalAbc));
-    checkSimplifyUnchanged(eq(literalAbc, literalZero));
-    checkSimplifyUnchanged(ne(literalZero, literalAbc));
-    checkSimplifyUnchanged(ne(literalAbc, literalZero));
-    checkSimplifyUnchanged(gt(literalZero, literalAbc));
-    checkSimplifyUnchanged(gt(literalAbc, literalZero));
-    checkSimplifyUnchanged(ge(literalZero, literalAbc));
-    checkSimplifyUnchanged(ge(literalAbc, literalZero));
-    checkSimplifyUnchanged(lt(literalZero, literalAbc));
-    checkSimplifyUnchanged(lt(literalAbc, literalZero));
-    checkSimplifyUnchanged(le(literalZero, literalAbc));
-    checkSimplifyUnchanged(le(literalAbc, literalZero));
+    checkSimplifyUnchanged(eq(literal(0), literalAbc));
+    checkSimplifyUnchanged(eq(literalAbc, literal(0)));
+    checkSimplifyUnchanged(ne(literal(0), literalAbc));
+    checkSimplifyUnchanged(ne(literalAbc, literal(0)));
+    checkSimplifyUnchanged(gt(literal(0), literalAbc));
+    checkSimplifyUnchanged(gt(literalAbc, literal(0)));
+    checkSimplifyUnchanged(ge(literal(0), literalAbc));
+    checkSimplifyUnchanged(ge(literalAbc, literal(0)));
+    checkSimplifyUnchanged(lt(literal(0), literalAbc));
+    checkSimplifyUnchanged(lt(literalAbc, literal(0)));
+    checkSimplifyUnchanged(le(literal(0), literalAbc));
+    checkSimplifyUnchanged(le(literalAbc, literal(0)));
   }
 
   /** Unit test for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-2421">[CALCITE-2421]
    * to-be-filled </a>. */
-  @Test void testSelfComparisions() {
+  @Test void testSelfComparisons() {
     checkSimplify3(and(eq(vInt(), vInt()), eq(vInt(1), vInt(1))),
         "AND(OR(null, IS NOT NULL(?0.int0)), OR(null, IS NOT NULL(?0.int1)))",
         "AND(IS NOT NULL(?0.int0), IS NOT NULL(?0.int1))",
@@ -2134,7 +2092,7 @@ class RexProgramTest extends RexProgramTestBase {
         "AND(IS NULL(?0.int0), IS NULL(?0.int1))");
   }
 
-  @Test void testBooleanComparisions() {
+  @Test void testBooleanComparisons() {
     checkSimplify(eq(vBool(), trueLiteral), "?0.bool0");
     checkSimplify(ge(vBool(), trueLiteral), "?0.bool0");
     checkSimplify(ne(vBool(), trueLiteral), "NOT(?0.bool0)");
@@ -2222,13 +2180,11 @@ class RexProgramTest extends RexProgramTestBase {
     final RexNode eRef = rexBuilder.makeFieldAccess(range, 4);
     final RexNode fRef = rexBuilder.makeFieldAccess(range, 5);
     final RexNode gRef = rexBuilder.makeFieldAccess(range, 6);
-    final RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
-    final RexLiteral literal2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(2));
 
     final ImmutableMap<RexNode, RexNode> map =
         RexUtil.predicateConstants(RexNode.class, rexBuilder,
             ImmutableList.of(eq(aRef, bRef),
-                eq(cRef, literal1),
+                eq(cRef, literal(1)),
                 eq(cRef, aRef),
                 eq(dRef, eRef)));
     assertThat(getString(map),
@@ -2238,16 +2194,16 @@ class RexProgramTest extends RexProgramTestBase {
     final RexNode ref0 = rexBuilder.makeInputRef(rowType, 0);
     final ImmutableMap<RexNode, RexNode> map2 =
         RexUtil.predicateConstants(RexNode.class, rexBuilder,
-            ImmutableList.of(eq(ref0, literal1),
-                eq(ref0, literal2)));
+            ImmutableList.of(eq(ref0, literal(1)),
+                eq(ref0, literal(2))));
     assertThat(getString(map2), is("{}"));
 
     // Contradictory constraints on field accesses SHOULD yield no constants
     // but currently there's a bug
     final ImmutableMap<RexNode, RexNode> map3 =
         RexUtil.predicateConstants(RexNode.class, rexBuilder,
-            ImmutableList.of(eq(aRef, literal1),
-                eq(aRef, literal2)));
+            ImmutableList.of(eq(aRef, literal(1)),
+                eq(aRef, literal(2))));
     assertThat(getString(map3), is("{1=?0.a, 2=?0.a}"));
 
     // Different precision and scale in decimal
@@ -2274,7 +2230,8 @@ class RexProgramTest extends RexProgramTestBase {
     final ImmutableMap<RexNode, RexNode> map6 =
         RexUtil.predicateConstants(RexNode.class, rexBuilder,
             ImmutableList.of(
-                eq(cast(eRef, typeFactory.createSqlType(SqlTypeName.INTEGER)), literal1)));
+                eq(cast(eRef, typeFactory.createSqlType(SqlTypeName.INTEGER)),
+                    literal(1))));
     assertThat(
         getString(map6), is("{1=CAST(?0.e):INTEGER NOT NULL, CAST(?0.e):INTEGER NOT NULL=1}"));
 
@@ -2282,7 +2239,8 @@ class RexProgramTest extends RexProgramTestBase {
     final ImmutableMap<RexNode, RexNode> map7 =
         RexUtil.predicateConstants(RexNode.class, rexBuilder,
             ImmutableList.of(
-                eq(cast(aRef, typeFactory.createSqlType(SqlTypeName.BIGINT)), literal1)));
+                eq(cast(aRef, typeFactory.createSqlType(SqlTypeName.BIGINT)),
+                    literal(1))));
     assertThat(getString(map7), is("{1=CAST(?0.a):BIGINT NOT NULL, ?0.a=1}"));
   }
 


[calcite] 02/03: [CALCITE-4170] Improve simplification of "<>" predicates

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 ca6c590718599e0b3420757b775994e1075843cc
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Sun Aug 9 15:58:48 2020 -0700

    [CALCITE-4170] Improve simplification of "<>" predicates
    
    We can now simplify 'a = 1 or a <> 1' to 'true'. To achieve
    this, we simplify the second term in a context where we know
    'a = 1' is false. This entails a RangeSet inside
    simplifyUsingPredicates, so that we can represent <> the same
    way that we represent =, > etc. All of the others are a Range,
    but <> requires a two-entry RangeSet ('a < 1 or a > 1').
---
 .../java/org/apache/calcite/rex/RexSimplify.java   | 37 +----------
 .../java/org/apache/calcite/util/RangeSets.java    | 46 +++++++++++++
 .../org/apache/calcite/rex/RexProgramTest.java     | 77 +++++++++++++++++++---
 .../apache/calcite/test/SqlToRelConverterTest.java |  4 +-
 .../java/org/apache/calcite/util/UtilTest.java     | 42 ++++++++++++
 .../apache/calcite/test/SqlToRelConverterTest.xml  | 13 ++--
 core/src/test/resources/sql/some.iq                |  2 +-
 7 files changed, 168 insertions(+), 53 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
index 5f94eb1..ee99936 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -32,6 +32,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.RangeSets;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ArrayListMultimap;
@@ -1577,48 +1578,12 @@ public class RexSimplify {
     return RexUtil.composeConjunction(rexBuilder, terms);
   }
 
-  private RexNode simplifyNotEqual(RexNode e) {
-    final Comparison comparison = Comparison.of(e);
-    if (comparison == null) {
-      return e;
-    }
-
-    for (RexNode node: predicates.pulledUpPredicates) {
-      final Comparison predicate = Comparison.of(node);
-      if (predicate == null
-          || predicate.kind != SqlKind.EQUALS
-          || !predicate.ref.equals(comparison.ref)) {
-        continue;
-      }
-
-      // Given x=5, x!=5 can be simplified to 'null and x is null' and x!=3 can
-      // be simplified to 'null or x is not null'.
-      RexNode simplified;
-      if (predicate.literal.equals(comparison.literal)) {
-        simplified = rexBuilder.makeCall(SqlStdOperatorTable.AND,
-            rexBuilder.makeNullLiteral(e.getType()),
-            rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, comparison.ref));
-      } else {
-        simplified = rexBuilder.makeCall(SqlStdOperatorTable.OR,
-            rexBuilder.makeNullLiteral(e.getType()),
-            rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, comparison.ref));
-      }
-      return simplify(simplified);
-    }
-
-    return e;
-  }
-
   private <C extends Comparable<C>> RexNode simplifyUsingPredicates(RexNode e,
       Class<C> clazz) {
     if (predicates.pulledUpPredicates.isEmpty()) {
       return e;
     }
 
-    if (e.getKind() == SqlKind.NOT_EQUALS) {
-      return simplifyNotEqual(e);
-    }
-
     final Comparison comparison = Comparison.of(e);
     // Check for comparison with null values
     if (comparison == null
diff --git a/core/src/main/java/org/apache/calcite/util/RangeSets.java b/core/src/main/java/org/apache/calcite/util/RangeSets.java
new file mode 100644
index 0000000..13de53a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/util/RangeSets.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.util;
+
+import com.google.common.collect.ImmutableRangeSet;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeSet;
+import com.google.common.collect.TreeRangeSet;
+
+/** Utilities for Guava {@link com.google.common.collect.RangeSet}. */
+@SuppressWarnings({"UnstableApiUsage"})
+public class RangeSets {
+  private RangeSets() {}
+
+  private static final ImmutableRangeSet ALL =
+      ImmutableRangeSet.of().complement();
+
+  /** Subtracts a range from a range set. */
+  public static <C extends Comparable<C>> RangeSet<C> minus(RangeSet<C> rangeSet,
+      Range<C> range) {
+    final TreeRangeSet<C> mutableRangeSet = TreeRangeSet.create(rangeSet);
+    mutableRangeSet.remove(range);
+    return mutableRangeSet.equals(rangeSet) ? rangeSet
+        : ImmutableRangeSet.copyOf(mutableRangeSet);
+  }
+
+  /** Returns the unrestricted range set. */
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  public static <C extends Comparable<C>> RangeSet<C> rangeSetAll() {
+    return (RangeSet) ALL;
+  }
+}
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 8582007..88db229 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
@@ -1354,27 +1354,86 @@ class RexProgramTest extends RexProgramTestBase {
     final RexNode aRef = rexBuilder.makeFieldAccess(range, 0);
     final RexNode bRef = rexBuilder.makeFieldAccess(range, 1);
     final RexNode cRef = rexBuilder.makeFieldAccess(range, 2);
+    final RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
+    final RexLiteral literal2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(2));
+    final RexLiteral literal3 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(3));
+    final RexLiteral literal4 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(4));
+    final RexLiteral literal5 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(5));
 
-    // "a != 1 or a = 1" ==> "true"
+    // "a <> 1 or a = 1" ==> "true"
     checkSimplifyFilter(
-        or(ne(aRef, literal(1)),
-            eq(aRef, literal(1))),
+        or(ne(aRef, literal1),
+            eq(aRef, literal1)),
         "true");
 
-    // TODO: make this simplify to "true"
+    // "a = 1 or a <> 1" ==> "true"
     checkSimplifyFilter(
-        or(eq(aRef, literal(1)),
-            ne(aRef, literal(1))),
-        "OR(=(?0.a, 1), <>(?0.a, 1))");
+        or(eq(aRef, literal1),
+            ne(aRef, literal1)),
+        "true");
+
+    // "a = 1 or a <> 2" could (and should) be simplified to "a <> 2"
+    // but can't do that right now
+    checkSimplifyFilter(
+        or(eq(aRef, literal1),
+            ne(aRef, literal2)),
+        "OR(=(?0.a, 1), <>(?0.a, 2))");
+
+    // "(a >= 1 and a <= 3) or a <> 2", or equivalently
+    // "a between 1 and 3 or a <> 2" ==> "true"
+    checkSimplifyFilter(
+        or(
+            and(ge(aRef, literal1),
+                le(aRef, literal3)),
+            ne(aRef, literal2)),
+        "true");
+
+    // "(a >= 1 and a <= 3) or a < 4" ==> "a < 4"
+    checkSimplifyFilter(
+        or(
+            and(ge(aRef, literal1),
+                le(aRef, literal3)),
+            lt(aRef, literal4)),
+        "<(?0.a, 4)");
+
+    // "(a >= 1 and a <= 2) or (a >= 4 and a <= 5) or a <> 3" ==> "a <> 3"
+    checkSimplifyFilter(
+        or(
+            and(ge(aRef, literal1),
+                le(aRef, literal2)),
+            and(ge(aRef, literal4),
+                le(aRef, literal5)),
+            ne(aRef, literal3)),
+        "<>(?0.a, 3)");
+
+    // "(a >= 1 and a <= 2) or (a >= 4 and a <= 5) or a <> 4" ==> "true"
+    checkSimplifyFilter(
+        or(
+            and(ge(aRef, literal1),
+                le(aRef, literal2)),
+            and(ge(aRef, literal4),
+                le(aRef, literal5)),
+            ne(aRef, literal4)),
+        "true");
 
-    // "b != 1 or b = 1" cannot be simplified, because b might be null
+    // "(a >= 1 and a <= 2) or (a > 4 and a <= 5) or a <> 4" ==> "a <> 4"
+    checkSimplifyFilter(
+        or(
+            and(ge(aRef, literal1),
+                le(aRef, literal2)),
+            and(gt(aRef, literal4),
+                le(aRef, literal5)),
+            ne(aRef, literal4)),
+        "<>(?0.a, 4)");
+
+    // "b <> 1 or b = 1" cannot be simplified, because b might be null
     final RexNode neOrEq =
         or(ne(bRef, literal(1)),
             eq(bRef, literal(1)));
     checkSimplifyFilter(neOrEq, "OR(<>(?0.b, 1), =(?0.b, 1))");
 
     // Careful of the excluded middle!
-    // We cannot simplify "b != 1 or b = 1" to "true" because if b is null, the
+    // 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.
     final RexNode simplified =
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 44124a6..5b6aa21 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -1859,8 +1859,8 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
 
   @Test void testTableFunctionTumbleWithOffset() {
     final String sql = "select *\n"
-        + "from table(tumble(table Shipments, descriptor(rowtime), INTERVAL '10' MINUTE, \n"
-        + "INTERVAL '1' MINUTE))";
+        + "from table(tumble(table Shipments, descriptor(rowtime),\n"
+        + "  INTERVAL '10' MINUTE, INTERVAL '1' MINUTE))";
     sql(sql).ok();
   }
 
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index 627ad36..0d88ff7 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -40,9 +40,12 @@ import org.apache.calcite.testlib.annotations.LocaleEnUs;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMultiset;
+import com.google.common.collect.ImmutableRangeSet;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeSet;
 import com.google.common.primitives.Ints;
 
 import org.hamcrest.Description;
@@ -2404,6 +2407,45 @@ class UtilTest {
     assertThat(map.containsKey("zyMurgy", false), is(true));
   }
 
+  /** Tests {@link RangeSets#minus(RangeSet, Range)}. */
+  @SuppressWarnings("UnstableApiUsage")
+  @Test void testRangeSetMinus() {
+    final RangeSet<Integer> setNone = ImmutableRangeSet.of();
+    final RangeSet<Integer> setAll = setNone.complement();
+    final RangeSet<Integer> setGt2 = ImmutableRangeSet.of(Range.greaterThan(2));
+    final RangeSet<Integer> setGt1 = ImmutableRangeSet.of(Range.greaterThan(1));
+    final RangeSet<Integer> setGe1 = ImmutableRangeSet.of(Range.atLeast(1));
+    final RangeSet<Integer> setGt0 = ImmutableRangeSet.of(Range.greaterThan(0));
+    final RangeSet<Integer> setComplex =
+        ImmutableRangeSet.<Integer>builder()
+            .add(Range.closed(0, 2))
+            .add(Range.singleton(3))
+            .add(Range.greaterThan(5))
+            .build();
+    assertThat(setComplex.toString(), is("[[0‥2], [3‥3], (5‥+∞)]"));
+
+    assertThat(RangeSets.minus(setAll, Range.singleton(1)).toString(),
+        is("[(-∞‥1), (1‥+∞)]"));
+    assertThat(RangeSets.minus(setNone, Range.singleton(1)), is(setNone));
+    assertThat(RangeSets.minus(setGt2, Range.singleton(1)), is(setGt2));
+    assertThat(RangeSets.minus(setGt1, Range.singleton(1)), is(setGt1));
+    assertThat(RangeSets.minus(setGe1, Range.singleton(1)), is(setGt1));
+    assertThat(RangeSets.minus(setGt0, Range.singleton(1)).toString(),
+        is("[(0‥1), (1‥+∞)]"));
+    assertThat(RangeSets.minus(setComplex, Range.singleton(1)).toString(),
+        is("[[0‥1), (1‥2], [3‥3], (5‥+∞)]"));
+    assertThat(RangeSets.minus(setComplex, Range.singleton(2)).toString(),
+        is("[[0‥2), [3‥3], (5‥+∞)]"));
+    assertThat(RangeSets.minus(setComplex, Range.singleton(3)).toString(),
+        is("[[0‥2], (5‥+∞)]"));
+    assertThat(RangeSets.minus(setComplex, Range.open(2, 3)).toString(),
+        is("[[0‥2], [3‥3], (5‥+∞)]"));
+    assertThat(RangeSets.minus(setComplex, Range.closed(2, 3)).toString(),
+        is("[[0‥2), (5‥+∞)]"));
+    assertThat(RangeSets.minus(setComplex, Range.closed(2, 7)).toString(),
+        is("[[0‥2), (7‥+∞)]"));
+  }
+
   @Test void testNlsStringClone() {
     final NlsString s = new NlsString("foo", "LATIN1", SqlCollation.IMPLICIT);
     assertThat(s.toString(), is("_LATIN1'foo'"));
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 69c57e2..7fe6df1 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -1046,7 +1046,9 @@ LogicalProject(ZIP=[$3])
 ]]>
         </Resource>
         <Resource name="sql">
-            <![CDATA[select empa.home_address.zip from sales.emp_address empa where empa.home_address.city = 'abc']]>
+            <![CDATA[select empa.home_address.zip
+from sales.emp_address empa
+where empa.home_address.city = 'abc']]>
         </Resource>
     </TestCase>
     <TestCase name="testSelectNestedColumnType">
@@ -2656,7 +2658,7 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EMPNO=[$0], EXPR$1=[OR(=($9, 0), AND(<($10, $9), null, <>($9, 0), IS NULL($13)), AND(<>($9, 0), IS NULL($13), >=($10, $9)))])
+LogicalProject(EMPNO=[$0], EXPR$1=[OR(=($9, 0), AND(<($10, $9), null, IS NULL($13)), AND(IS NULL($13), >=($10, $9)))])
   LogicalJoin(condition=[=($11, $12)], joinType=[left])
     LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f0=[$9], $f1=[$10], DEPTNO0=[$7])
       LogicalJoin(condition=[true], joinType=[inner])
@@ -2800,7 +2802,7 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EMPNO=[$0], EXPR$1=[OR(=($9, 0), AND(<($10, $9), null, <>($9, 0), IS NULL($13)), AND(<>($9, 0), IS NULL($13), >=($10, $9)))])
+LogicalProject(EMPNO=[$0], EXPR$1=[OR(=($9, 0), AND(<($10, $9), null, IS NULL($13)), AND(IS NULL($13), >=($10, $9)))])
   LogicalJoin(condition=[=($11, $12)], joinType=[left])
     LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f0=[$9], $f1=[$10], DEPTNO0=[$7])
       LogicalJoin(condition=[true], joinType=[inner])
@@ -3763,7 +3765,7 @@ LogicalProject(DEPTNO=[$7])
     </TestCase>
     <TestCase name="testReduceConstExpr">
         <Resource name="sql">
-            <![CDATA[select sum(case when 'y' = 'n' then ename else 1 end) from emp]]>
+            <![CDATA[select sum(case when 'y' = 'n' then ename else 0.1 end) from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -5078,7 +5080,8 @@ LogicalProject(ORDERID=[$0], ROWTIME=[$1], window_start=[$2], window_end=[$3], O
     <TestCase name="testTableFunctionTumbleWithOffset">
         <Resource name="sql">
             <![CDATA[select *
-from table(tumble(table Shipments, descriptor(rowtime), INTERVAL '10' MINUTE, INTERVAL '1' MINUTE))]]>
+from table(tumble(table Shipments, descriptor(rowtime),
+  INTERVAL '10' MINUTE, INTERVAL '1' MINUTE))]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
diff --git a/core/src/test/resources/sql/some.iq b/core/src/test/resources/sql/some.iq
index 8ed4cf3..083256f 100644
--- a/core/src/test/resources/sql/some.iq
+++ b/core/src/test/resources/sql/some.iq
@@ -133,7 +133,7 @@ from "scott".emp;
 (14 rows)
 
 !ok
-EnumerableCalc(expr#0..10=[{inputs}], expr#11=[0], expr#12=[=($t9, $t11)], expr#13=[>($t9, $t10)], expr#14=[null:BOOLEAN], expr#15=[<>($t9, $t11)], expr#16=[<=($t5, $t8)], expr#17=[IS NOT TRUE($t16)], expr#18=[AND($t13, $t14, $t15, $t17)], expr#19=[>($t5, $t8)], expr#20=[<=($t9, $t10)], expr#21=[AND($t19, $t15, $t17, $t20)], expr#22=[OR($t12, $t18, $t21)], proj#0..7=[{exprs}], X=[$t22])
+EnumerableCalc(expr#0..10=[{inputs}], expr#11=[0], expr#12=[=($t9, $t11)], expr#13=[>($t9, $t10)], expr#14=[null:BOOLEAN], expr#15=[<=($t5, $t8)], expr#16=[IS NOT TRUE($t15)], expr#17=[AND($t13, $t14, $t16)], expr#18=[>($t5, $t8)], expr#19=[<=($t9, $t10)], expr#20=[AND($t18, $t16, $t19)], expr#21=[OR($t12, $t17, $t20)], proj#0..7=[{exprs}], X=[$t21])
   EnumerableNestedLoopJoin(condition=[true], joinType=[inner])
     EnumerableTableScan(table=[[scott, EMP]])
     EnumerableAggregate(group=[{}], m=[MAX($6)], c=[COUNT()], d=[COUNT($6)])