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 2018/05/29 06:26:02 UTC

calcite git commit: [CALCITE-2314] Verify RexNode transformations by evaluating before and after expressions against sample values

Repository: calcite
Updated Branches:
  refs/heads/master c37880565 -> 6fe55e3ec


[CALCITE-2314] Verify RexNode transformations by evaluating before and after expressions against sample values

Add RexInterpreter, a simple (and incomplete, but good enough for
testing) interpreter of RexNode expressions.

Interpreter skips expressions involving CAST.

Print NullSentinel.INSTANCE as "NULL"

Disable test for simplifying CEIL/FLOOR (logged CALCITE-2332)

Fix bugs in RexInterpreter. (Zoltan Haindrich)


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/6fe55e3e
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/6fe55e3e
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/6fe55e3e

Branch: refs/heads/master
Commit: 6fe55e3ecc1f476bb0d629007f128ee62e771ff0
Parents: c378805
Author: Julian Hyde <jh...@apache.org>
Authored: Wed May 16 13:34:16 2018 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon May 28 21:36:57 2018 -0700

----------------------------------------------------------------------
 .../calcite/rel/metadata/NullSentinel.java      |   6 +-
 .../org/apache/calcite/rex/RexAnalyzer.java     | 142 ++++++++
 .../org/apache/calcite/rex/RexInterpreter.java  | 354 +++++++++++++++++++
 .../java/org/apache/calcite/rex/RexLiteral.java |  21 ++
 .../org/apache/calcite/rex/RexSimplify.java     | 128 +++++--
 .../main/java/org/apache/calcite/util/Util.java |  20 ++
 .../calcite/test/MaterializationTest.java       |   2 +-
 .../calcite/test/RexImplicationCheckerTest.java |   6 +-
 .../org/apache/calcite/test/RexProgramTest.java |  78 ++--
 9 files changed, 710 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/6fe55e3e/core/src/main/java/org/apache/calcite/rel/metadata/NullSentinel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/NullSentinel.java b/core/src/main/java/org/apache/calcite/rel/metadata/NullSentinel.java
index 4215211..2f023ec 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/NullSentinel.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/NullSentinel.java
@@ -19,7 +19,11 @@ package org.apache.calcite.rel.metadata;
 /** Placeholder for null values. */
 public enum NullSentinel {
   /** Placeholder for a null value. */
-  INSTANCE,
+  INSTANCE {
+    @Override public String toString() {
+      return "NULL";
+    }
+  },
 
   /** Placeholder that means that a request for metadata is already active,
    * therefore this request forms a cycle. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/6fe55e3e/core/src/main/java/org/apache/calcite/rex/RexAnalyzer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexAnalyzer.java b/core/src/main/java/org/apache/calcite/rex/RexAnalyzer.java
new file mode 100644
index 0000000..c080ccf
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rex/RexAnalyzer.java
@@ -0,0 +1,142 @@
+/*
+ * 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.rex;
+
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.rel.metadata.NullSentinel;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+
+import java.math.BigDecimal;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/** Analyzes an expression, figures out what are the unbound variables,
+ * assigns a variety of values to each unbound variable, and evaluates
+ * the expression. */
+public class RexAnalyzer {
+  public final RexNode e;
+  public final List<RexNode> variables;
+  public final int unsupportedCount;
+
+  /** Creates a RexAnalyzer. */
+  public RexAnalyzer(RexNode e, RelOptPredicateList predicates) {
+    this.e = e;
+    final VariableCollector variableCollector = new VariableCollector();
+    e.accept(variableCollector);
+    predicates.pulledUpPredicates.forEach(p -> p.accept(variableCollector));
+    variables = ImmutableList.copyOf(variableCollector.builder);
+    unsupportedCount = variableCollector.unsupportedCount;
+  }
+
+  /** Generates a map of variables and lists of values that could be assigned
+   * to them. */
+  public Iterable<Map<RexNode, Comparable>> assignments() {
+    final List<List<Comparable>> generators =
+        variables.stream().map(RexAnalyzer::getComparables)
+            .collect(Util.toImmutableList());
+    final Iterable<List<Comparable>> product = Linq4j.product(generators);
+    //noinspection StaticPseudoFunctionalStyleMethod
+    return Iterables.transform(product,
+        values -> ImmutableMap.copyOf(Pair.zip(variables, values)));
+  }
+
+  private static List<Comparable> getComparables(RexNode variable) {
+    final ImmutableList.Builder<Comparable> values = ImmutableList.builder();
+    switch (variable.getType().getSqlTypeName()) {
+    case BOOLEAN:
+      values.add(true);
+      values.add(false);
+      break;
+    case INTEGER:
+      values.add(BigDecimal.valueOf(-1L));
+      values.add(BigDecimal.valueOf(0L));
+      values.add(BigDecimal.valueOf(1L));
+      values.add(BigDecimal.valueOf(1_000_000L));
+      break;
+    case VARCHAR:
+      values.add(new NlsString("", null, null));
+      values.add(new NlsString("hello", null, null));
+      break;
+    case TIMESTAMP:
+      values.add(0L); // 1970-01-01 00:00:00
+      break;
+    case DATE:
+      values.add(0); // 1970-01-01
+      values.add(365); // 1971-01-01
+      values.add(-365); // 1969-01-01
+      break;
+    case TIME:
+      values.add(0); // 00:00:00.000
+      values.add(86_399_000); // 23:59:59.000
+      break;
+    default:
+      throw new AssertionError("don't know values for " + variable
+          + " of type " + variable.getType());
+    }
+    if (variable.getType().isNullable()) {
+      values.add(NullSentinel.INSTANCE);
+    }
+    return values.build();
+  }
+
+  /** Collects the variables (or other bindable sites) in an expression, and
+   * counts features (such as CAST) that {@link RexInterpreter} cannot
+   * handle. */
+  private static class VariableCollector extends RexVisitorImpl<Void> {
+    private final Set<RexNode> builder = new LinkedHashSet<>();
+    private int unsupportedCount = 0;
+
+    VariableCollector() {
+      super(true);
+    }
+
+    @Override public Void visitInputRef(RexInputRef inputRef) {
+      builder.add(inputRef);
+      return super.visitInputRef(inputRef);
+    }
+
+    @Override public Void visitFieldAccess(RexFieldAccess fieldAccess) {
+      if (fieldAccess.getReferenceExpr() instanceof RexDynamicParam) {
+        builder.add(fieldAccess);
+        return null;
+      } else {
+        return super.visitFieldAccess(fieldAccess);
+      }
+    }
+
+    @Override public Void visitCall(RexCall call) {
+      switch (call.getKind()) {
+      case CAST:
+        ++unsupportedCount;
+        return null;
+      default:
+        return super.visitCall(call);
+      }
+    }
+  }
+}
+
+// End RexAnalyzer.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/6fe55e3e/core/src/main/java/org/apache/calcite/rex/RexInterpreter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexInterpreter.java b/core/src/main/java/org/apache/calcite/rex/RexInterpreter.java
new file mode 100644
index 0000000..84c3a7c
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rex/RexInterpreter.java
@@ -0,0 +1,354 @@
+/*
+ * 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.rex;
+
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.rel.metadata.NullSentinel;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableMap;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.IntPredicate;
+
+/**
+ * Evaluates {@link RexNode} expressions.
+ *
+ * <p>Caveats:
+ * <ul>
+ *   <li>It uses interpretation, so it is not very efficient.
+ *   <li>It is intended for testing, so does not cover very many functions and
+ *   operators. (Feel free to contribute more!)
+ *   <li>It is not well tested.
+ * </ul>
+ */
+public class RexInterpreter implements RexVisitor<Comparable> {
+  private static final NullSentinel N = NullSentinel.INSTANCE;
+
+  private final Map<RexNode, Comparable> environment;
+
+  /** Creates an interpreter.
+   *
+   * @param environment Values of certain expressions (usually
+   *       {@link RexInputRef}s)
+   */
+  private RexInterpreter(Map<RexNode, Comparable> environment) {
+    this.environment = ImmutableMap.copyOf(environment);
+  }
+
+  /** Evaluates an expression in an environment. */
+  public static Comparable evaluate(RexNode e, Map<RexNode, Comparable> map) {
+    final Comparable v = e.accept(new RexInterpreter(map));
+    if (false) {
+      System.out.println("evaluate " + e + " on " + map + " returns " + v);
+    }
+    return v;
+  }
+
+  private IllegalArgumentException unbound(RexNode e) {
+    return new IllegalArgumentException("unbound: " + e);
+  }
+
+  private Comparable getOrUnbound(RexNode e) {
+    final Comparable comparable = environment.get(e);
+    if (comparable != null) {
+      return comparable;
+    }
+    throw unbound(e);
+  }
+
+  public Comparable visitInputRef(RexInputRef inputRef) {
+    return getOrUnbound(inputRef);
+  }
+
+  public Comparable visitLocalRef(RexLocalRef localRef) {
+    throw unbound(localRef);
+  }
+
+  public Comparable visitLiteral(RexLiteral literal) {
+    return Util.first(literal.getValue4(), NullSentinel.INSTANCE);
+  }
+
+  public Comparable visitOver(RexOver over) {
+    throw unbound(over);
+  }
+
+  public Comparable visitCorrelVariable(RexCorrelVariable correlVariable) {
+    return getOrUnbound(correlVariable);
+  }
+
+  public Comparable visitDynamicParam(RexDynamicParam dynamicParam) {
+    return getOrUnbound(dynamicParam);
+  }
+
+  public Comparable visitRangeRef(RexRangeRef rangeRef) {
+    throw unbound(rangeRef);
+  }
+
+  public Comparable visitFieldAccess(RexFieldAccess fieldAccess) {
+    return getOrUnbound(fieldAccess);
+  }
+
+  public Comparable visitSubQuery(RexSubQuery subQuery) {
+    throw unbound(subQuery);
+  }
+
+  public Comparable visitTableInputRef(RexTableInputRef fieldRef) {
+    throw unbound(fieldRef);
+  }
+
+  public Comparable visitPatternFieldRef(RexPatternFieldRef fieldRef) {
+    throw unbound(fieldRef);
+  }
+
+  public Comparable visitCall(RexCall call) {
+    final List<Comparable> values = new ArrayList<>(call.operands.size());
+    for (RexNode operand : call.operands) {
+      values.add(operand.accept(this));
+    }
+    switch (call.getKind()) {
+    case EQUALS:
+      return compare(values, c -> c == 0);
+    case NOT_EQUALS:
+      return compare(values, c -> c != 0);
+    case GREATER_THAN:
+      return compare(values, c -> c > 0);
+    case GREATER_THAN_OR_EQUAL:
+      return compare(values, c -> c >= 0);
+    case LESS_THAN:
+      return compare(values, c -> c < 0);
+    case LESS_THAN_OR_EQUAL:
+      return compare(values, c -> c <= 0);
+    case AND:
+      return values.stream().map(Truthy::of).min(Comparator.naturalOrder())
+          .get().toComparable();
+    case OR:
+      return values.stream().map(Truthy::of).max(Comparator.naturalOrder())
+          .get().toComparable();
+    case NOT:
+      return not(values.get(0));
+    case CASE:
+      return case_(values);
+    case IS_TRUE:
+      return values.get(0).equals(true);
+    case IS_NOT_TRUE:
+      return !values.get(0).equals(true);
+    case IS_NULL:
+      return values.get(0).equals(N);
+    case IS_NOT_NULL:
+      return !values.get(0).equals(N);
+    case IS_FALSE:
+      return values.get(0).equals(false);
+    case IS_NOT_FALSE:
+      return !values.get(0).equals(false);
+    case PLUS:
+      return containsNull(values) ? N
+          : number(values.get(0)).add(number(values.get(1)));
+    case CAST:
+      return cast(call, values);
+    case COALESCE:
+      return coalesce(call, values);
+    case CEIL:
+    case FLOOR:
+      return ceil(call, values);
+    case EXTRACT:
+      return extract(call, values);
+    default:
+      throw unbound(call);
+    }
+  }
+
+  private Comparable extract(RexCall call, List<Comparable> values) {
+    final Comparable v = values.get(1);
+    if (v == N) {
+      return N;
+    }
+    final TimeUnitRange timeUnitRange = (TimeUnitRange) values.get(0);
+    final int v2;
+    if (v instanceof Long) {
+      // TIMESTAMP
+      v2 = (int) (((Long) v) / TimeUnit.DAY.multiplier.longValue());
+    } else {
+      // DATE
+      v2 = (Integer) v;
+    }
+    return DateTimeUtils.unixDateExtract(timeUnitRange, v2);
+  }
+
+  private Comparable coalesce(RexCall call, List<Comparable> values) {
+    for (Comparable value : values) {
+      if (value != N) {
+        return value;
+      }
+    }
+    return N;
+  }
+
+  private Comparable ceil(RexCall call, List<Comparable> values) {
+    if (values.get(0) == N) {
+      return N;
+    }
+    final Long v = (Long) values.get(0);
+    final TimeUnitRange unit = (TimeUnitRange) values.get(1);
+    switch (unit) {
+    case YEAR:
+    case MONTH:
+      switch (call.getKind()) {
+      case FLOOR:
+        return DateTimeUtils.unixTimestampFloor(unit, v);
+      default:
+        return DateTimeUtils.unixTimestampCeil(unit, v);
+      }
+    }
+    final TimeUnitRange subUnit = subUnit(unit);
+    for (long v2 = v;;) {
+      final int e = DateTimeUtils.unixTimestampExtract(subUnit, v2);
+      if (e == 0) {
+        return v2;
+      }
+      v2 -= unit.startUnit.multiplier.longValue();
+    }
+  }
+
+  private TimeUnitRange subUnit(TimeUnitRange unit) {
+    switch (unit) {
+    case QUARTER:
+      return TimeUnitRange.MONTH;
+    default:
+      return TimeUnitRange.DAY;
+    }
+  }
+
+  private Comparable cast(RexCall call, List<Comparable> values) {
+    if (values.get(0) == N) {
+      return N;
+    }
+    return values.get(0);
+  }
+
+  private Comparable not(Comparable value) {
+    if (value.equals(true)) {
+      return false;
+    } else if (value.equals(false)) {
+      return true;
+    } else {
+      return N;
+    }
+  }
+
+  private Comparable case_(List<Comparable> values) {
+    final int size;
+    final Comparable elseValue;
+    if (values.size() % 2 == 0) {
+      size = values.size();
+      elseValue = N;
+    } else {
+      size = values.size() - 1;
+      elseValue = Util.last(values);
+    }
+    for (int i = 0; i < size; i += 2) {
+      if (values.get(i).equals(true)) {
+        return values.get(i + 1);
+      }
+    }
+    return elseValue;
+  }
+
+  private BigDecimal number(Comparable comparable) {
+    return comparable instanceof BigDecimal
+        ? (BigDecimal) comparable
+        : comparable instanceof BigInteger
+        ? new BigDecimal((BigInteger) comparable)
+            : comparable instanceof Long
+                || comparable instanceof Integer
+                || comparable instanceof Short
+        ? new BigDecimal(((Number) comparable).longValue())
+        : new BigDecimal(((Number) comparable).doubleValue());
+  }
+
+  private Comparable compare(List<Comparable> values, IntPredicate p) {
+    if (containsNull(values)) {
+      return N;
+    }
+    Comparable v0 = values.get(0);
+    Comparable v1 = values.get(1);
+
+    if (v0 instanceof Number && v1 instanceof NlsString) {
+      try {
+        v1 = new BigDecimal(((NlsString) v1).getValue());
+      } catch (NumberFormatException e) {
+        return false;
+      }
+    }
+    if (v1 instanceof Number && v0 instanceof NlsString) {
+      try {
+        v0 = new BigDecimal(((NlsString) v0).getValue());
+      } catch (NumberFormatException e) {
+        return false;
+      }
+    }
+    if (v0 instanceof Number) {
+      v0 = number(v0);
+    }
+    if (v1 instanceof Number) {
+      v1 = number(v1);
+    }
+    //noinspection unchecked
+    final int c = v0.compareTo(v1);
+    return p.test(c);
+  }
+
+  private boolean containsNull(List<Comparable> values) {
+    for (Comparable value : values) {
+      if (value == NullSentinel.INSTANCE) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** An enum that wraps boolean and unknown values and makes them
+   * comparable. */
+  enum Truthy {
+    // Order is important; AND returns the min, OR returns the max
+    FALSE, UNKNOWN, TRUE;
+
+    static Truthy of(Comparable c) {
+      return c.equals(true) ? TRUE : c.equals(false) ? FALSE : UNKNOWN;
+    }
+
+    Comparable toComparable() {
+      switch (this) {
+      case TRUE: return true;
+      case FALSE: return false;
+      case UNKNOWN: return N;
+      default:
+        throw new AssertionError();
+      }
+    }
+  }
+}
+
+// End RexInterpreter.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/6fe55e3e/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
index 9143616..94ea886 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
@@ -764,6 +764,27 @@ public class RexLiteral extends RexNode {
     }
   }
 
+  /**
+   * Returns the value of this literal, in the form that {@link RexInterpreter}
+   * wants it.
+   */
+  public Comparable getValue4() {
+    if (value == null) {
+      return null;
+    }
+    switch (typeName) {
+    case TIMESTAMP:
+    case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+      return getValueAs(Long.class);
+    case DATE:
+    case TIME:
+    case TIME_WITH_LOCAL_TIME_ZONE:
+      return getValueAs(Integer.class);
+    default:
+      return value;
+    }
+  }
+
   /** Returns the value of this literal as an instance of the specified class.
    *
    * <p>The following SQL types allow more than one form:

http://git-wip-us.apache.org/repos/asf/calcite/blob/6fe55e3e/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
----------------------------------------------------------------------
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 7befc4d..d168402 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -23,6 +23,7 @@ import org.apache.calcite.plan.RelOptPredicateList;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.Strong;
 import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.metadata.NullSentinel;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
@@ -47,11 +48,13 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.Function;
 
 /**
  * Context required to simplify a row-expression.
  */
 public class RexSimplify {
+  private final boolean paranoid;
   public final RexBuilder rexBuilder;
   private final RelOptPredicateList predicates;
   final boolean unknownAsFalse;
@@ -67,9 +70,16 @@ public class RexSimplify {
    */
   public RexSimplify(RexBuilder rexBuilder, RelOptPredicateList predicates,
       boolean unknownAsFalse, RexExecutor executor) {
+    this(rexBuilder, predicates, unknownAsFalse, false, executor);
+  }
+
+  /** Internal constructor. */
+  private RexSimplify(RexBuilder rexBuilder, RelOptPredicateList predicates,
+      boolean unknownAsFalse, boolean paranoid, RexExecutor executor) {
     this.rexBuilder = Preconditions.checkNotNull(rexBuilder);
     this.predicates = Preconditions.checkNotNull(predicates);
     this.unknownAsFalse = unknownAsFalse;
+    this.paranoid = paranoid;
     this.executor = Preconditions.checkNotNull(executor);
   }
 
@@ -86,7 +96,8 @@ public class RexSimplify {
   public RexSimplify withUnknownAsFalse(boolean unknownAsFalse) {
     return unknownAsFalse == this.unknownAsFalse
         ? this
-        : new RexSimplify(rexBuilder, predicates, unknownAsFalse, executor);
+        : new RexSimplify(rexBuilder, predicates, unknownAsFalse, paranoid,
+            executor);
   }
 
   /** Returns a RexSimplify the same as this but with a specified
@@ -94,7 +105,20 @@ public class RexSimplify {
   public RexSimplify withPredicates(RelOptPredicateList predicates) {
     return predicates == this.predicates
         ? this
-        : new RexSimplify(rexBuilder, predicates, unknownAsFalse, executor);
+        : new RexSimplify(rexBuilder, predicates, unknownAsFalse, paranoid,
+            executor);
+  }
+
+  /** Returns a RexSimplify the same as this but which verifies that
+   * the expression before and after simplification are equivalent.
+   *
+   * @see #verify
+   */
+  public RexSimplify withParanoid(boolean paranoid) {
+    return paranoid == this.paranoid
+        ? this
+        : new RexSimplify(rexBuilder, predicates, unknownAsFalse, paranoid,
+            executor);
   }
 
   /** Simplifies a boolean expression, always preserving its type and its
@@ -133,6 +157,10 @@ public class RexSimplify {
    * @param e Expression to simplify
    */
   public RexNode simplify(RexNode e) {
+    return verify(e, simplifier -> simplifier.simplify_(e));
+  }
+
+  private RexNode simplify_(RexNode e) {
     switch (e.getKind()) {
     case AND:
       return simplifyAnd((RexCall) e);
@@ -193,7 +221,7 @@ public class RexSimplify {
       case GREATER_THAN_OR_EQUAL:
       case LESS_THAN_OR_EQUAL:
         // "x = x" simplifies to "x is not null" (similarly <= and >=)
-        return simplify(
+        return simplify_(
             rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, o0));
       default:
         // "x != x" simplifies to "false" (similarly < and >)
@@ -264,7 +292,7 @@ public class RexSimplify {
 
   private void simplifyList(List<RexNode> terms) {
     for (int i = 0; i < terms.size(); i++) {
-      terms.set(i, withUnknownAsFalse(false).simplify(terms.get(i)));
+      terms.set(i, withUnknownAsFalse(false).simplify_(terms.get(i)));
     }
   }
 
@@ -322,17 +350,17 @@ public class RexSimplify {
     switch (a.getKind()) {
     case NOT:
       // NOT NOT x ==> x
-      return simplify(((RexCall) a).getOperands().get(0));
+      return simplify_(((RexCall) a).getOperands().get(0));
     }
     final SqlKind negateKind = a.getKind().negate();
     if (a.getKind() != negateKind) {
-      return simplify(
+      return simplify_(
           rexBuilder.makeCall(RexUtil.op(negateKind),
               ImmutableList.of(((RexCall) a).getOperands().get(0))));
     }
     final SqlKind negateKind2 = a.getKind().negateNullSafe();
     if (a.getKind() != negateKind2) {
-      return simplify(
+      return simplify_(
           rexBuilder.makeCall(RexUtil.op(negateKind2),
               ((RexCall) a).getOperands()));
     }
@@ -341,18 +369,19 @@ public class RexSimplify {
       final List<RexNode> newOperands = new ArrayList<>();
       for (RexNode operand : ((RexCall) a).getOperands()) {
         newOperands.add(
-            simplify(rexBuilder.makeCall(SqlStdOperatorTable.NOT, operand)));
+            simplify_(rexBuilder.makeCall(SqlStdOperatorTable.NOT, operand)));
       }
-      return simplify(rexBuilder.makeCall(SqlStdOperatorTable.OR, newOperands));
+      return simplify_(
+          rexBuilder.makeCall(SqlStdOperatorTable.OR, newOperands));
     }
     if (a.getKind() == SqlKind.OR) {
       // NOT distributivity for OR
       final List<RexNode> newOperands = new ArrayList<>();
       for (RexNode operand : ((RexCall) a).getOperands()) {
         newOperands.add(
-            simplify(rexBuilder.makeCall(SqlStdOperatorTable.NOT, operand)));
+            simplify_(rexBuilder.makeCall(SqlStdOperatorTable.NOT, operand)));
       }
-      return simplify(
+      return simplify_(
           rexBuilder.makeCall(SqlStdOperatorTable.AND, newOperands));
     }
     return call;
@@ -413,7 +442,7 @@ public class RexSimplify {
       // x IS TRUE ==> x (if x is not nullable)
       // x IS NOT FALSE ==> x (if x is not nullable)
       if (!a.getType().isNullable()) {
-        return simplify(a);
+        return simplify_(a);
       }
       break;
     case IS_FALSE:
@@ -421,7 +450,7 @@ public class RexSimplify {
       // 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()) {
-        return simplify(rexBuilder.makeCall(SqlStdOperatorTable.NOT, a));
+        return simplify_(rexBuilder.makeCall(SqlStdOperatorTable.NOT, a));
       }
       break;
     }
@@ -435,9 +464,9 @@ public class RexSimplify {
       // because of null values.
       final SqlOperator notKind = RexUtil.op(kind.negateNullSafe());
       final RexNode arg = ((RexCall) a).operands.get(0);
-      return simplify(rexBuilder.makeCall(notKind, arg));
+      return simplify_(rexBuilder.makeCall(notKind, arg));
     }
-    RexNode a2 = simplify(a);
+    RexNode a2 = simplify_(a);
     if (a != a2) {
       return rexBuilder.makeCall(RexUtil.op(kind), ImmutableList.of(a2));
     }
@@ -484,7 +513,7 @@ public class RexSimplify {
     final Set<String> digests = new HashSet<>();
     final List<RexNode> operands = new ArrayList<>();
     for (RexNode operand : call.getOperands()) {
-      operand = simplify(operand);
+      operand = simplify_(operand);
       if (digests.add(operand.digest)) {
         operands.add(operand);
       }
@@ -637,6 +666,7 @@ public class RexSimplify {
     return builder.build();
   }
 
+  // public only to support a deprecated method; treat as private
   public RexNode simplifyAnd(RexCall e) {
     final List<RexNode> terms = new ArrayList<>();
     final List<RexNode> notTerms = new ArrayList<>();
@@ -656,6 +686,7 @@ public class RexSimplify {
     return simplifyAnd2(terms, notTerms);
   }
 
+  // package-protected only to support a deprecated method; treat as private
   RexNode simplifyAnd2(List<RexNode> terms, List<RexNode> notTerms) {
     for (RexNode term : terms) {
       if (term.isAlwaysFalse()) {
@@ -681,7 +712,7 @@ public class RexSimplify {
     // Add the NOT disjunctions back in.
     for (RexNode notDisjunction : notTerms) {
       terms.add(
-          simplify(
+          simplify_(
               rexBuilder.makeCall(SqlStdOperatorTable.NOT, notDisjunction)));
     }
     return RexUtil.composeConjunction(rexBuilder, terms, false);
@@ -707,7 +738,7 @@ public class RexSimplify {
     }
     if (terms.size() == 1 && notTerms.isEmpty()) {
       // Make sure "x OR y OR x" (a single-term conjunction) gets simplified.
-      return simplify(terms.get(0));
+      return simplify_(terms.get(0));
     }
     // Try to simplify the expression
     final Multimap<String, Pair<String, RexNode>> equalityTerms = ArrayListMultimap.create();
@@ -907,7 +938,7 @@ public class RexSimplify {
     for (RexNode notDisjunction : notTerms) {
       final RexNode call =
           rexBuilder.makeCall(SqlStdOperatorTable.NOT, notDisjunction);
-      terms.add(simplify(call));
+      terms.add(simplify_(call));
     }
     // The negated terms: only deterministic expressions
     for (String negatedTerm : negatedTerms) {
@@ -1015,8 +1046,12 @@ public class RexSimplify {
   /** Simplifies a list of terms and combines them into an OR.
    * Modifies the list in place. */
   public RexNode simplifyOrs(List<RexNode> terms) {
+    if (paranoid) {
+      final RexNode before = RexUtil.composeDisjunction(rexBuilder, terms);
+      return verify(before, simplifier -> simplifier.simplifyOrs(terms));
+    }
     for (int i = 0; i < terms.size(); i++) {
-      final RexNode term = simplify(terms.get(i));
+      final RexNode term = simplify_(terms.get(i));
       switch (term.getKind()) {
       case LITERAL:
         if (RexLiteral.isNullLiteral(term)) {
@@ -1040,6 +1075,57 @@ public class RexSimplify {
     return RexUtil.composeDisjunction(rexBuilder, terms);
   }
 
+  private RexNode verify(RexNode before,
+      Function<RexSimplify, RexNode> simplifier) {
+    final RexNode simplified = simplifier.apply(withParanoid(false));
+    if (!paranoid) {
+      return simplified;
+    }
+    final RexAnalyzer foo0 = new RexAnalyzer(before, predicates);
+    final RexAnalyzer foo1 = new RexAnalyzer(simplified, predicates);
+    if (foo0.unsupportedCount > 0 || foo1.unsupportedCount > 0) {
+      // Analyzer cannot handle this expression currently
+      return simplified;
+    }
+    if (!foo0.variables.containsAll(foo1.variables)) {
+      throw new AssertionError("variable mismatch: "
+          + before + " has " + foo0.variables + ", "
+          + simplified + " has " + foo1.variables);
+    }
+    assignment_loop:
+    for (Map<RexNode, Comparable> map : foo0.assignments()) {
+      for (RexNode predicate : predicates.pulledUpPredicates) {
+        final Comparable v = RexInterpreter.evaluate(predicate, map);
+        if (!v.equals(true)) {
+          continue assignment_loop;
+        }
+      }
+      Comparable v0 = RexInterpreter.evaluate(foo0.e, map);
+      if (v0 == null) {
+        throw new AssertionError("interpreter returned null for " + foo0.e);
+      }
+      Comparable v1 = RexInterpreter.evaluate(foo1.e, map);
+      if (v1 == null) {
+        throw new AssertionError("interpreter returned null for " + foo1.e);
+      }
+      if (unknownAsFalse
+          && before.getType().getSqlTypeName() == SqlTypeName.BOOLEAN) {
+        if (v0 == NullSentinel.INSTANCE) {
+          v0 = false;
+        }
+        if (v1 == NullSentinel.INSTANCE) {
+          v1 = false;
+        }
+      }
+      if (!v0.equals(v1)) {
+        throw new AssertionError("result mismatch: when applied to " + map
+            + ", " + before + " yielded " + v0
+            + ", and " + simplified + " yielded " + v1);
+      }
+    }
+    return simplified;
+  }
+
   private RexNode simplifyCast(RexCall e) {
     final RexNode operand = e.getOperands().get(0);
     switch (operand.getKind()) {
@@ -1092,7 +1178,7 @@ public class RexSimplify {
       // Bail out since we only simplify ceil/floor <date>
       return e;
     }
-    final RexNode operand = simplify(e.getOperands().get(0));
+    final RexNode operand = simplify_(e.getOperands().get(0));
     switch (operand.getKind()) {
     case CEIL:
     case FLOOR:

http://git-wip-us.apache.org/repos/asf/calcite/blob/6fe55e3e/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
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 64ab61f..307879c 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -97,6 +97,7 @@ import java.util.TimeZone;
 import java.util.jar.JarFile;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+import java.util.stream.Collector;
 import javax.annotation.Nullable;
 
 import static com.google.common.base.Preconditions.checkNotNull;
@@ -2346,6 +2347,25 @@ public class Util {
     return calendar;
   }
 
+  /**
+   * Returns a {@code Collector} that accumulates the input elements into a
+   * Guava {@link ImmutableList} via a {@link ImmutableList.Builder}.
+   *
+   * @param <T> Type of the input elements
+   *
+   * @return a {@code Collector} that collects all the input elements into an
+   * {@link ImmutableList}, in encounter order
+   */
+  public static <T> Collector<T, ImmutableList.Builder<T>, ImmutableList<T>>
+      toImmutableList() {
+    return Collector.of(ImmutableList::builder, ImmutableList.Builder::add,
+        (t, u) -> {
+          t.addAll(u.build());
+          return t;
+        },
+        ImmutableList.Builder::build);
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/6fe55e3e/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
index 4f7905c..e1b314c 100644
--- a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
@@ -122,7 +122,7 @@ public class MaterializationTest {
   private final RexBuilder rexBuilder = new RexBuilder(typeFactory);
   private final RexSimplify simplify =
       new RexSimplify(rexBuilder, RelOptPredicateList.EMPTY, false,
-          RexUtil.EXECUTOR);
+          RexUtil.EXECUTOR).withParanoid(true);
 
   @Test public void testScan() {
     CalciteAssert.that()

http://git-wip-us.apache.org/repos/asf/calcite/blob/6fe55e3e/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java b/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
index 9520e45..24b78c2 100644
--- a/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
@@ -50,6 +50,7 @@ import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.math.BigDecimal;
@@ -377,6 +378,9 @@ public class RexImplicationCheckerTest {
   }
 
   /** Test case for simplifier of ceil and floor. */
+  // Disabled: we wrongly simplify FLOOR(CEIL(2010-10-10, YEAR), YEAR)
+  // to FLOOR(2010-10-10, YEAR)
+  @Ignore("[CALCITE-2332]")
   @Test public void testSimplifyFloor() {
     final ImmutableList<TimeUnitRange> timeUnitRanges =
         ImmutableList.of(TimeUnitRange.WEEK,
@@ -521,7 +525,7 @@ public class RexImplicationCheckerTest {
       executor = holder.get();
       simplify =
           new RexSimplify(rexBuilder, RelOptPredicateList.EMPTY, false,
-              executor);
+              executor).withParanoid(true);
       checker = new RexImplicationChecker(rexBuilder, executor, rowType);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/6fe55e3e/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
index 90b56bc..007b1c4 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -24,6 +24,7 @@ import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.plan.RelOptPredicateList;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.Strong;
+import org.apache.calcite.rel.metadata.NullSentinel;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
@@ -33,6 +34,7 @@ import org.apache.calcite.rex.RexDynamicParam;
 import org.apache.calcite.rex.RexExecutor;
 import org.apache.calcite.rex.RexExecutorImpl;
 import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexInterpreter;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
@@ -112,7 +114,8 @@ public class RexProgramTest {
     RexExecutor executor =
         new RexExecutorImpl(new DummyTestDataContext());
     simplify =
-        new RexSimplify(rexBuilder, RelOptPredicateList.EMPTY, false, executor);
+        new RexSimplify(rexBuilder, RelOptPredicateList.EMPTY, false, executor)
+            .withParanoid(true);
     trueLiteral = rexBuilder.makeLiteral(true);
     falseLiteral = rexBuilder.makeLiteral(false);
     final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
@@ -1109,6 +1112,8 @@ public class RexProgramTest {
         .add("g", booleanType)
         .add("h", intType)
         .add("i", intNullableType)
+        .add("j", intType)
+        .add("k", intType)
         .build();
 
     final RexDynamicParam range = rexBuilder.makeDynamicParam(rowType, 0);
@@ -1119,6 +1124,8 @@ public class RexProgramTest {
     final RexNode eRef = rexBuilder.makeFieldAccess(range, 4);
     final RexNode hRef = rexBuilder.makeFieldAccess(range, 7);
     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
@@ -1208,36 +1215,36 @@ public class RexProgramTest {
         "true");
 
     // condition, and the inverse - nothing to do due to null values
-    checkSimplify2(and(le(aRef, literal1), gt(aRef, literal1)),
-        "AND(<=(?0.a, 1), >(?0.a, 1))",
+    checkSimplify2(and(le(hRef, literal1), gt(hRef, literal1)),
+        "AND(<=(?0.h, 1), >(?0.h, 1))",
         "false");
 
-    checkSimplify2(and(le(aRef, literal1), ge(aRef, literal1)),
-        "AND(<=(?0.a, 1), >=(?0.a, 1))",
-        "=(?0.a, 1)");
+    checkSimplify2(and(le(hRef, literal1), ge(hRef, literal1)),
+        "AND(<=(?0.h, 1), >=(?0.h, 1))",
+        "=(?0.h, 1)");
 
-    checkSimplify2(and(lt(aRef, literal1), eq(aRef, literal1), ge(aRef, literal1)),
-        "AND(<(?0.a, 1), =(?0.a, 1), >=(?0.a, 1))",
+    checkSimplify2(and(lt(hRef, literal1), eq(hRef, literal1), ge(hRef, literal1)),
+        "AND(<(?0.h, 1), =(?0.h, 1), >=(?0.h, 1))",
         "false");
 
-    checkSimplify(and(lt(aRef, literal1), or(falseLiteral, falseLiteral)),
+    checkSimplify(and(lt(hRef, literal1), or(falseLiteral, falseLiteral)),
         "false");
-    checkSimplify(and(lt(aRef, literal1), or(falseLiteral, gt(bRef, cRef))),
-        "AND(<(?0.a, 1), >(?0.b, ?0.c))");
-    checkSimplify(or(lt(aRef, literal1), and(trueLiteral, trueLiteral)),
+    checkSimplify(and(lt(hRef, literal1), or(falseLiteral, gt(jRef, kRef))),
+        "AND(<(?0.h, 1), >(?0.j, ?0.k))");
+    checkSimplify(or(lt(hRef, literal1), and(trueLiteral, trueLiteral)),
         "true");
     checkSimplify(
-        or(lt(aRef, literal1),
+        or(lt(hRef, literal1),
             and(trueLiteral, or(trueLiteral, falseLiteral))),
         "true");
     checkSimplify(
-        or(lt(aRef, literal1),
+        or(lt(hRef, literal1),
             and(trueLiteral, and(trueLiteral, falseLiteral))),
-        "<(?0.a, 1)");
+        "<(?0.h, 1)");
     checkSimplify(
-        or(lt(aRef, literal1),
+        or(lt(hRef, literal1),
             and(trueLiteral, or(falseLiteral, falseLiteral))),
-        "<(?0.a, 1)");
+        "<(?0.h, 1)");
 
     // "x = x" simplifies to "x is not null"
     checkSimplify(eq(literal1, literal1), "true");
@@ -1290,8 +1297,8 @@ public class RexProgramTest {
         typeFactory.createSqlType(SqlTypeName.BOOLEAN);
     final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
     final RelDataType rowType = typeFactory.builder()
-        .add("a", booleanType)
-        .add("b", booleanType)
+        .add("a", intType)
+        .add("b", intType)
         .add("c", booleanType)
         .add("d", booleanType)
         .add("e", booleanType)
@@ -1305,6 +1312,8 @@ public class RexProgramTest {
     final RexNode bRef = rexBuilder.makeFieldAccess(range, 1);
     final RexNode cRef = rexBuilder.makeFieldAccess(range, 2);
     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);
@@ -1360,8 +1369,8 @@ public class RexProgramTest {
 
     // case: trailing false and null, remove
     checkSimplifyFilter(
-        case_(aRef, trueLiteral, bRef, trueLiteral, cRef, falseLiteral, dRef, falseLiteral,
-            unknownLiteral), "CAST(OR(?0.a, ?0.b)):BOOLEAN");
+        case_(cRef, trueLiteral, dRef, trueLiteral, eRef, falseLiteral, fRef,
+            falseLiteral, unknownLiteral), "CAST(OR(?0.c, ?0.d)):BOOLEAN");
 
     // condition with null value for range
     checkSimplifyFilter(and(gt(aRef, unknownLiteral), ge(bRef, literal1)), "false");
@@ -2139,8 +2148,31 @@ public class RexProgramTest {
   }
 
   private RexNode simplify(RexNode e) {
-    return new RexSimplify(rexBuilder, RelOptPredicateList.EMPTY, false,
-        RexUtil.EXECUTOR).simplify(e);
+    final RexSimplify simplify =
+        new RexSimplify(rexBuilder, RelOptPredicateList.EMPTY, false,
+            RexUtil.EXECUTOR).withParanoid(true);
+    return simplify.simplify(e);
+  }
+
+  @Test public void testInterpreter() {
+    assertThat(eval(trueLiteral), is(true));
+    assertThat(eval(nullLiteral), is(NullSentinel.INSTANCE));
+    assertThat(eval(eq(nullLiteral, nullLiteral)),
+        is(NullSentinel.INSTANCE));
+    assertThat(eval(eq(this.trueLiteral, nullLiteral)),
+        is(NullSentinel.INSTANCE));
+    assertThat(eval(eq(falseLiteral, trueLiteral)),
+        is(false));
+    assertThat(eval(ne(falseLiteral, trueLiteral)),
+        is(true));
+    assertThat(eval(ne(falseLiteral, nullLiteral)),
+        is(NullSentinel.INSTANCE));
+    assertThat(eval(and(this.trueLiteral, falseLiteral)),
+        is(false));
+  }
+
+  private Comparable eval(RexNode e) {
+    return RexInterpreter.evaluate(e, ImmutableMap.of());
   }
 }