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 2015/07/17 02:10:14 UTC

[2/3] incubator-calcite git commit: Fix up previous commit; add some tests for constant reduction

Fix up previous commit; add some tests for constant reduction


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

Branch: refs/heads/master
Commit: 5a365609c0d050e5e832449bdadad1e309f0f204
Parents: 4a9b193
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Jul 15 21:48:51 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Jul 16 15:03:20 2015 -0700

----------------------------------------------------------------------
 .../adapter/enumerable/EnumerableWindow.java    |   2 +-
 .../MaterializedViewSubstitutionVisitor.java    |  69 +--
 .../org/apache/calcite/plan/RelOptUtil.java     |  17 +-
 .../calcite/plan/RexImplicationChecker.java     | 256 ++++----
 .../calcite/plan/SubstitutionVisitor.java       |  53 +-
 .../apache/calcite/plan/VisitorDataContext.java |  57 +-
 .../rel/rules/AggregateFilterTransposeRule.java |   2 +-
 .../rel/rules/AggregateUnionTransposeRule.java  |   2 +-
 .../rel/rules/FilterAggregateTransposeRule.java |   2 +-
 .../java/org/apache/calcite/sql/SqlKind.java    |  20 +
 .../org/apache/calcite/rex/RexExecutorTest.java | 136 ++--
 .../org/apache/calcite/test/DiffRepository.java |   1 -
 .../calcite/test/RexImplicationCheckerTest.java | 614 ++++++++-----------
 13 files changed, 592 insertions(+), 639 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5a365609/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
index 15f77f5..433c6cd 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
@@ -746,7 +746,7 @@ public class EnumerableWindow extends Window implements EnumerableRel {
       BlockBuilder builder, final Result result, int windowIdx,
       List<AggImpState> aggs, PhysType outputPhysType,
       List<Expression> outputRow) {
-    for (final AggImpState agg: aggs) {
+    for (final AggImpState agg : aggs) {
       agg.context =
           new WinAggContext() {
             public SqlAggFunction aggregation() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5a365609/core/src/main/java/org/apache/calcite/plan/MaterializedViewSubstitutionVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/MaterializedViewSubstitutionVisitor.java b/core/src/main/java/org/apache/calcite/plan/MaterializedViewSubstitutionVisitor.java
index e367f93..5c4ccdd 100644
--- a/core/src/main/java/org/apache/calcite/plan/MaterializedViewSubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/MaterializedViewSubstitutionVisitor.java
@@ -25,45 +25,19 @@ import org.apache.calcite.rex.RexShuttle;
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
+
 /**
- * Substitutes part of a tree of relational expressions with another tree.
- *
- * <p>The call {@code new MaterializedSubstitutionVisitor(target, query).go(replacement))}
- * will return {@code query} with every occurrence of {@code target} replaced
- * by {@code replacement}.</p>
- *
- * <p>The following example shows how {@code MaterializedSubstitutionVisitor} can be used
- * for materialized view recognition.</p>
- *
- * <ul>
- * <li>query = SELECT a, c FROM t WHERE x = 5 AND b = 4</li>
- * <li>target = SELECT a, b, c FROM t WHERE x = 5</li>
- * <li>replacement = SELECT * FROM mv</li>
- * <li>result = SELECT a, c FROM mv WHERE b = 4</li>
- * </ul>
- *
- * <p>Note that {@code result} uses the materialized view table {@code mv} and a
- * simplified condition {@code b = 4}.</p>
- *
- * <p>Uses a bottom-up matching algorithm. Nodes do not need to be identical.
- * At each level, returns the residue.</p>
- *
- * <p>The inputs must only include the core relational operators:
- * {@link org.apache.calcite.rel.logical.LogicalTableScan},
- * {@link LogicalFilter},
- * {@link LogicalProject},
- * {@link org.apache.calcite.rel.logical.LogicalJoin},
- * {@link LogicalUnion},
- * {@link LogicalAggregate}.</p>
+ * Extension to {@link SubstitutionVisitor}.
  */
 public class MaterializedViewSubstitutionVisitor extends SubstitutionVisitor {
+  private static final ImmutableList<UnifyRule> EXTENDED_RULES =
+      ImmutableList.<UnifyRule>builder()
+          .addAll(DEFAULT_RULES)
+          .add(ProjectToProjectUnifyRule1.INSTANCE)
+          .build();
 
   public MaterializedViewSubstitutionVisitor(RelNode target_, RelNode query_) {
-    super(target_, query_);
-    ImmutableList.Builder<UnifyRule> builder = new ImmutableList.Builder<UnifyRule>();
-    builder.addAll(this.unifyRules);
-    builder.add(ProjectToProjectUnifyRule1.INSTANCE);
-    this.unifyRules = builder.build();
+    super(target_, query_, EXTENDED_RULES);
   }
 
   public RelNode go(RelNode replacement_) {
@@ -73,7 +47,6 @@ public class MaterializedViewSubstitutionVisitor extends SubstitutionVisitor {
   /**
    * Project to Project Unify rule.
    */
-
   private static class ProjectToProjectUnifyRule1 extends AbstractUnifyRule {
     public static final ProjectToProjectUnifyRule1 INSTANCE =
         new ProjectToProjectUnifyRule1();
@@ -83,12 +56,13 @@ public class MaterializedViewSubstitutionVisitor extends SubstitutionVisitor {
           operand(MutableProject.class, target(0)), 1);
     }
 
-    @Override
-    protected UnifyResult apply(UnifyRuleCall call) {
+    @Override protected UnifyResult apply(UnifyRuleCall call) {
       final MutableProject query = (MutableProject) call.query;
 
-      final List<RelDataTypeField> oldFieldList = query.getInput().getRowType().getFieldList();
-      final List<RelDataTypeField> newFieldList = call.target.getRowType().getFieldList();
+      final List<RelDataTypeField> oldFieldList =
+          query.getInput().getRowType().getFieldList();
+      final List<RelDataTypeField> newFieldList =
+          call.target.getRowType().getFieldList();
       List<RexNode> newProjects;
       try {
         newProjects = transformRex(query.getProjects(), oldFieldList, newFieldList);
@@ -104,9 +78,8 @@ public class MaterializedViewSubstitutionVisitor extends SubstitutionVisitor {
       return call.result(newProject2);
     }
 
-    @Override
-    protected UnifyRuleCall match(SubstitutionVisitor visitor, MutableRel query,
-        MutableRel target) {
+    @Override protected UnifyRuleCall match(SubstitutionVisitor visitor,
+        MutableRel query, MutableRel target) {
       assert query instanceof MutableProject && target instanceof MutableProject;
 
       if (queryOperand.matches(visitor, query)) {
@@ -116,8 +89,8 @@ public class MaterializedViewSubstitutionVisitor extends SubstitutionVisitor {
 
           final MutableProject queryProject = (MutableProject) query;
           if (queryProject.getInput() instanceof MutableFilter) {
-
-            final MutableFilter innerFilter = (MutableFilter) (queryProject.getInput());
+            final MutableFilter innerFilter =
+                (MutableFilter) queryProject.getInput();
             RexNode newCondition;
             try {
               newCondition = transformRex(innerFilter.getCondition(),
@@ -130,18 +103,18 @@ public class MaterializedViewSubstitutionVisitor extends SubstitutionVisitor {
                 newCondition);
 
             return visitor.new UnifyRuleCall(this, query, newFilter,
-                copy(visitor.getSlots(), slotCount));
+                copy(visitor.slots, slotCount));
           }
         }
       }
       return null;
     }
 
-    private RexNode transformRex(
-        RexNode node,
+    private RexNode transformRex(RexNode node,
         final List<RelDataTypeField> oldFields,
         final List<RelDataTypeField> newFields) {
-      List<RexNode> nodes = transformRex(ImmutableList.of(node), oldFields, newFields);
+      List<RexNode> nodes =
+          transformRex(ImmutableList.of(node), oldFields, newFields);
       return nodes.get(0);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5a365609/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
index c1f8af5..7a79aeb 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -1158,7 +1158,7 @@ public abstract class RelOptUtil {
             && kind != SqlKind.EQUALS
             && kind != SqlKind.IS_DISTINCT_FROM) {
           if (reverse) {
-            kind = reverse(kind);
+            kind = kind.reverse();
           }
           rangeOp.add(op(kind, call.getOperator()));
         }
@@ -1199,21 +1199,6 @@ public abstract class RelOptUtil {
         false);
   }
 
-  public static SqlKind reverse(SqlKind kind) {
-    switch (kind) {
-    case GREATER_THAN:
-      return SqlKind.LESS_THAN;
-    case GREATER_THAN_OR_EQUAL:
-      return SqlKind.LESS_THAN_OR_EQUAL;
-    case LESS_THAN:
-      return SqlKind.GREATER_THAN;
-    case LESS_THAN_OR_EQUAL:
-      return SqlKind.GREATER_THAN_OR_EQUAL;
-    default:
-      return kind;
-    }
-  }
-
   public static SqlOperator op(SqlKind kind, SqlOperator operator) {
     switch (kind) {
     case EQUALS:

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5a365609/core/src/main/java/org/apache/calcite/plan/RexImplicationChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RexImplicationChecker.java b/core/src/main/java/org/apache/calcite/plan/RexImplicationChecker.java
index 1eb019d..42c7b43 100644
--- a/core/src/main/java/org/apache/calcite/plan/RexImplicationChecker.java
+++ b/core/src/main/java/org/apache/calcite/plan/RexImplicationChecker.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.calcite.plan;
 
 import org.apache.calcite.DataContext;
@@ -39,40 +38,44 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-
-
-/** Checks if Condition X logically implies Condition Y
+/**
+ * Checks whether one condition logically implies another.
  *
- * <p>(x > 10) implies (x > 5)</p>
+ * <p>If A &rArr; B, whenever A is true, B will be true also.
  *
- * <p>(y = 10) implies (y < 30 AND x > 30)</p>
+ * <p>For example:
+ * <ul>
+ * <li>(x &gt; 10) &rArr; (x &gt; 5)
+ * <li>(y = 10) &rArr; (y &lt; 30 OR x &gt; 30)
+ * </ul>
  */
 public class RexImplicationChecker {
   final RexBuilder builder;
-  final RexExecutorImpl rexImpl;
+  final RexExecutorImpl executor;
   final RelDataType rowType;
 
   public RexImplicationChecker(
       RexBuilder builder,
-      RexExecutorImpl rexImpl,
+      RexExecutorImpl executor,
       RelDataType rowType) {
     this.builder = builder;
-    this.rexImpl = rexImpl;
+    this.executor = executor;
     this.rowType = rowType;
   }
 
   /**
-   * Checks if condition first implies (=>) condition second
-   * This reduces to SAT problem which is NP-Complete
-   * When func says first => second then it is definitely true
-   * It cannot prove if first doesnot imply second.
+   * Checks if condition first implies (&rArr;) condition second.
+   *
+   * <p>This reduces to SAT problem which is NP-Complete.
+   * When this method says first implies second then it is definitely true.
+   * But it cannot prove that first does not imply second.
+   *
    * @param first first condition
    * @param second second condition
-   * @return true if it can prove first => second, otherwise false i.e.,
-   * it doesn't know if implication holds .
+   * @return true if it can prove first &rArr; second; otherwise false i.e.,
+   * it doesn't know if implication holds
    */
   public boolean implies(RexNode first, RexNode second) {
-
     // Validation
     if (!validate(first, second)) {
       return false;
@@ -91,22 +94,26 @@ public class RexImplicationChecker {
       return true;
     }
 
-    /** Decompose DNF into List of Conjunctions
+    /** Decomposes DNF into List of Conjunctions.
      *
-     * (x > 10 AND y > 30) OR (z > 90) will be converted to
+     * <p>For example,
+     * {@code x > 10 AND y > 30) OR (z > 90)}
+     * will be converted to
      * list of 2 conditions:
-     * 1. (x > 10 AND y > 30)
-     * 2. (z > 90)
      *
+     * <ul>
+     *   <li>(x > 10 AND y > 30)</li>
+     *   <li>z > 90</li>
+     * </ul>
      */
     List<RexNode> firstDnfs = RelOptUtil.disjunctions(firstDnf);
     List<RexNode> secondDnfs = RelOptUtil.disjunctions(secondDnf);
 
     for (RexNode f : firstDnfs) {
       if (!f.isAlwaysFalse()) {
-        //Check if f implies atleast
+        // Check if f implies at least
         // one of the conjunctions in list secondDnfs
-        boolean implyOneConjuntion = false;
+        boolean implyOneConjunction = false;
         for (RexNode s : secondDnfs) {
           if (s.isAlwaysFalse()) { // f cannot imply s
             continue;
@@ -115,57 +122,61 @@ public class RexImplicationChecker {
           if (impliesConjunction(f, s)) {
             // Satisfies one of the condition, so lets
             // move to next conjunction in firstDnfs
-            implyOneConjuntion = true;
+            implyOneConjunction = true;
             break;
           }
-        } //end of inner loop
+        }
 
-        // If f couldnot imply even one conjunction in
+        // If f could not imply even one conjunction in
         // secondDnfs, then final implication may be false
-        if (!implyOneConjuntion) {
+        if (!implyOneConjunction) {
           return false;
         }
       }
-    } //end of outer loop
+    }
 
     return true;
   }
 
-  /** Checks if Conjunction first => Conjunction second**/
+  /** Returns whether first implies second (both are conjunctions). */
   private boolean impliesConjunction(RexNode first, RexNode second) {
+    final InputUsageFinder firstUsageFinder = new InputUsageFinder();
+    final InputUsageFinder secondUsageFinder = new InputUsageFinder();
 
-    InputUsageFinder firstUsgFinder = new InputUsageFinder();
-    InputUsageFinder secondUsgFinder = new InputUsageFinder();
-
-    RexUtil.apply(firstUsgFinder, new ArrayList<RexNode>(), first);
-    RexUtil.apply(secondUsgFinder, new ArrayList<RexNode>(), second);
+    RexUtil.apply(firstUsageFinder, new ArrayList<RexNode>(), first);
+    RexUtil.apply(secondUsageFinder, new ArrayList<RexNode>(), second);
 
     // Check Support
-    if (!checkSupport(firstUsgFinder, secondUsgFinder)) {
+    if (!checkSupport(firstUsageFinder, secondUsageFinder)) {
       return false;
     }
 
-    List<Pair<RexInputRef, RexNode>> usgList = new ArrayList<>();
-    for (Map.Entry<RexInputRef, InputRefUsage<SqlOperator,
-        RexNode>> entry: firstUsgFinder.usageMap.entrySet()) {
-      final List<Pair<SqlOperator, RexNode>> list = entry.getValue().getUsageList();
-      usgList.add(Pair.of(entry.getKey(), list.get(0).getValue()));
+    List<Pair<RexInputRef, RexNode>> usageList = new ArrayList<>();
+    for (Map.Entry<RexInputRef, InputRefUsage<SqlOperator, RexNode>> entry
+        : firstUsageFinder.usageMap.entrySet()) {
+      final Pair<SqlOperator, RexNode> pair = entry.getValue().usageList.get(0);
+      usageList.add(Pair.of(entry.getKey(), pair.getValue()));
     }
 
-    /* Get the literals from first conjunction and execute second conjunction using them
-     * E.g., for x >30 => x > 10,
-     * we will replace x by 30 in second expression and execute it i.e., 30>10
-     * If it's true then we infer implication.
-     */
-    final DataContext dataValues = VisitorDataContext.getDataContext(rowType, usgList);
+    // Get the literals from first conjunction and executes second conjunction
+    // using them.
+    //
+    // E.g., for
+    //   x > 30 &rArr; x > 10,
+    // we will replace x by 30 in second expression and execute it i.e.,
+    //   30 > 10
+    //
+    // If it's true then we infer implication.
+    final DataContext dataValues =
+        VisitorDataContext.of(rowType, usageList);
 
     if (dataValues == null) {
       return false;
     }
 
     ImmutableList<RexNode> constExps = ImmutableList.of(second);
-    final RexExecutable exec = rexImpl.getExecutable(builder,
-        constExps, rowType);
+    final RexExecutable exec =
+        executor.getExecutable(builder, constExps, rowType);
 
     Object[] result;
     exec.setDataContext(dataValues);
@@ -176,72 +187,85 @@ public class RexImplicationChecker {
       // Need to monitor it and handle all the cases raising them.
       return false;
     }
-    return result != null && result.length == 1 && result[0] instanceof Boolean
+    return result != null
+        && result.length == 1
+        && result[0] instanceof Boolean
         && (Boolean) result[0];
   }
 
   /**
    * Looks at the usage of variables in first and second conjunction to decide
-   * if this kind of expression is currently supported for proving first => second.
-   * 1. Variables should be used only once in both the conjunction against
-   *    given set of operations only: >,<,<=,>=,=,!=
-   * 2. All the variables used in second condition should be used even in the first.
-   * 3. If operator used for variable in first is op1 and op2 for second, then we support
-   *    these combination for conjunction (op1, op2) then op1, op2 belongs to
-   *    one of the following sets:
-   *    a. (<,<=) X (<,<=) , X represents cartesian product
-   *    b. (>/>=) X (>,>=)
-   *    c. (=) X (>,>=,<,<=,=,!=)
-   *    d. (!=, =)
-   * @return true, if input usage pattern is supported. Otherwise, false.
+   * whether this kind of expression is currently supported for proving first
+   * implies second.
+   *
+   * <ol>
+   * <li>Variables should be used only once in both the conjunction against
+   * given set of operations only: >, <, <=, >=, =, !=
+   *
+   * <li>All the variables used in second condition should be used even in the
+   * first.
+   *
+   * <li>If operator used for variable in first is op1 and op2 for second, then
+   * we support these combination for conjunction (op1, op2) then op1, op2
+   * belongs to one of the following sets:
+   *
+   * <ul>
+   *    <li>(<, <=) X (<, <=)      <i>note: X represents cartesian product</i>
+   *    <li>(> / >=) X (>, >=)
+   *    <li>(=) X (>, >=, <, <=, =, !=)
+   *    <li>(!=, =)
+   * </ul>
+   * </ol>
+   *
+   * @return whether input usage pattern is supported
    */
-  private boolean checkSupport(
-      InputUsageFinder firstUsgFinder,
-      InputUsageFinder secondUsgFinder) {
-    Map<RexInputRef, InputRefUsage<SqlOperator,
-        RexNode>> firstUsgMap = firstUsgFinder.usageMap;
-    Map<RexInputRef, InputRefUsage<SqlOperator,
-        RexNode>> secondUsgMap = secondUsgFinder.usageMap;
-
-    for (Map.Entry<RexInputRef, InputRefUsage<SqlOperator,
-        RexNode>> entry: firstUsgMap.entrySet()) {
+  private boolean checkSupport(InputUsageFinder firstUsageFinder,
+      InputUsageFinder secondUsageFinder) {
+    final Map<RexInputRef, InputRefUsage<SqlOperator, RexNode>> firstUsageMap =
+        firstUsageFinder.usageMap;
+    final Map<RexInputRef, InputRefUsage<SqlOperator, RexNode>> secondUsageMap =
+        secondUsageFinder.usageMap;
+
+    for (Map.Entry<RexInputRef, InputRefUsage<SqlOperator, RexNode>> entry
+        : firstUsageMap.entrySet()) {
       if (entry.getValue().usageCount > 1) {
         return false;
       }
     }
 
-    for (Map.Entry<RexInputRef, InputRefUsage<SqlOperator,
-        RexNode>> entry: secondUsgMap.entrySet()) {
+    for (Map.Entry<RexInputRef, InputRefUsage<SqlOperator, RexNode>> entry
+        : secondUsageMap.entrySet()) {
       final InputRefUsage<SqlOperator, RexNode> secondUsage = entry.getValue();
-      if (secondUsage.getUsageCount() > 1
-          || secondUsage.getUsageList().size() != 1) {
+      if (secondUsage.usageCount > 1
+          || secondUsage.usageList.size() != 1) {
         return false;
       }
 
-      final InputRefUsage<SqlOperator, RexNode> firstUsage = firstUsgMap.get(entry.getKey());
+      final InputRefUsage<SqlOperator, RexNode> firstUsage =
+          firstUsageMap.get(entry.getKey());
       if (firstUsage == null
-          || firstUsage.getUsageList().size() != 1) {
+          || firstUsage.usageList.size() != 1) {
         return false;
       }
 
-      final Pair<SqlOperator, RexNode> fUse = firstUsage.getUsageList().get(0);
-      final Pair<SqlOperator, RexNode> sUse = secondUsage.getUsageList().get(0);
+      final Pair<SqlOperator, RexNode> fUse = firstUsage.usageList.get(0);
+      final Pair<SqlOperator, RexNode> sUse = secondUsage.usageList.get(0);
 
-      final SqlKind fkind = fUse.getKey().getKind();
+      final SqlKind fKind = fUse.getKey().getKind();
 
-      if (fkind != SqlKind.EQUALS) {
+      if (fKind != SqlKind.EQUALS) {
         switch (sUse.getKey().getKind()) {
         case GREATER_THAN:
         case GREATER_THAN_OR_EQUAL:
-          if (!(fkind == SqlKind.GREATER_THAN)
-              && !(fkind == SqlKind.GREATER_THAN_OR_EQUAL)) {
+          if (!(fKind == SqlKind.GREATER_THAN)
+              && !(fKind == SqlKind.GREATER_THAN_OR_EQUAL)) {
             return false;
           }
           break;
         case LESS_THAN:
         case LESS_THAN_OR_EQUAL:
-          if (!(fkind == SqlKind.LESS_THAN)
-              && !(fkind == SqlKind.LESS_THAN_OR_EQUAL)) {
+          if (!(fKind == SqlKind.LESS_THAN)
+              && !(fKind == SqlKind.LESS_THAN_OR_EQUAL)) {
             return false;
           }
           break;
@@ -254,35 +278,29 @@ public class RexImplicationChecker {
   }
 
   private boolean validate(RexNode first, RexNode second) {
-    if (first == null || second == null) {
-      return false;
-    }
-    if (!(first instanceof RexCall)
-        || !(second instanceof RexCall)) {
-      return false;
-    }
-    return true;
+    return first instanceof RexCall && second instanceof RexCall;
   }
 
-
   /**
-   * Visitor which builds a Usage Map of inputs used by an expression.
-   * E.g: for x >10 AND y < 20 AND x =40, Usage Map would look like:
-   * key:x value: {(>,10),(=,40), usageCount = 2}
-   * key:y value: {(>,20),usageCount=1}
+   * Visitor that builds a usage map of inputs used by an expression.
+   *
+   * <p>E.g: for x > 10 AND y < 20 AND x = 40, usage map is as follows:
+   * <ul>
+   * <li>key: x value: {(>, 10),(=, 40), usageCount = 2}
+   * <li>key: y value: {(>, 20), usageCount = 1}
+   * </ul>
    */
   private static class InputUsageFinder extends RexVisitorImpl<Void> {
-    public final Map<RexInputRef, InputRefUsage<SqlOperator,
-        RexNode>> usageMap = new HashMap<>();
+    public final Map<RexInputRef, InputRefUsage<SqlOperator, RexNode>>
+    usageMap = new HashMap<>();
 
     public InputUsageFinder() {
       super(true);
     }
 
     public Void visitInputRef(RexInputRef inputRef) {
-      InputRefUsage<SqlOperator,
-          RexNode> inputRefUse = getUsageMap(inputRef);
-      inputRefUse.incrUsage();
+      InputRefUsage<SqlOperator, RexNode> inputRefUse = getUsageMap(inputRef);
+      inputRefUse.usageCount++;
       return null;
     }
 
@@ -318,8 +336,7 @@ public class RexImplicationChecker {
     }
 
     private SqlOperator reverse(SqlOperator op) {
-      return RelOptUtil.op(
-          RelOptUtil.reverse(op.getKind()), op);
+      return RelOptUtil.op(op.getKind().reverse(), op);
     }
 
     private static RexNode removeCast(RexNode inputRef) {
@@ -333,11 +350,12 @@ public class RexImplicationChecker {
       return inputRef;
     }
 
-    private void updateUsage(SqlOperator op, RexInputRef inputRef, RexNode literal) {
-      InputRefUsage<SqlOperator,
-          RexNode> inputRefUse = getUsageMap(inputRef);
+    private void updateUsage(SqlOperator op, RexInputRef inputRef,
+        RexNode literal) {
+      final InputRefUsage<SqlOperator, RexNode> inputRefUse =
+          getUsageMap(inputRef);
       Pair<SqlOperator, RexNode> use = Pair.of(op, literal);
-      inputRefUse.getUsageList().add(use);
+      inputRefUse.usageList.add(use);
     }
 
     private InputRefUsage<SqlOperator, RexNode> getUsageMap(RexInputRef rex) {
@@ -352,26 +370,12 @@ public class RexImplicationChecker {
   }
 
   /**
-   * DataStructure to store usage of InputRefs in expression
+   * Usage of a {@link RexInputRef} in an expression.
    */
-
   private static class InputRefUsage<T1, T2> {
-    private final List<Pair<T1, T2>> usageList =
-        new ArrayList<Pair<T1, T2>>();
+    private final List<Pair<T1, T2>> usageList = new ArrayList<>();
     private int usageCount = 0;
-
-    public InputRefUsage() {}
-
-    public int getUsageCount() {
-      return usageCount;
-    }
-
-    public void incrUsage() {
-      usageCount++;
-    }
-
-    public List<Pair<T1, T2>> getUsageList() {
-      return usageList;
-    }
   }
 }
+
+// End RexImplicationChecker.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5a365609/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
index 62b2d9d..14e836a 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -145,11 +145,19 @@ public class SubstitutionVisitor {
   private static final Equivalence<List<?>> PAIRWISE_STRING_EQUIVALENCE =
       (Equivalence) STRING_EQUIVALENCE.pairwise();
 
-  protected static List<UnifyRule> unifyRules;
+  protected static final ImmutableList<UnifyRule> DEFAULT_RULES =
+      ImmutableList.<UnifyRule>of(
+//          TrivialRule.INSTANCE,
+          ProjectToProjectUnifyRule.INSTANCE,
+          FilterToProjectUnifyRule.INSTANCE,
+//          ProjectToFilterUnifyRule.INSTANCE,
+          FilterToFilterUnifyRule.INSTANCE,
+          AggregateToAggregateUnifyRule.INSTANCE,
+          AggregateOnProjectToAggregateUnifyRule.INSTANCE);
 
-  private static final Map<Pair<Class, Class>, List<UnifyRule>> RULE_MAP =
+  private final ImmutableList<UnifyRule> rules;
+  private final Map<Pair<Class, Class>, List<UnifyRule>> ruleMap =
       new HashMap<>();
-
   private final RelOptCluster cluster;
   private final Holder query;
   private final MutableRel target;
@@ -174,8 +182,16 @@ public class SubstitutionVisitor {
    * Assumes no rule needs more than 2 slots. */
   protected final MutableRel[] slots = new MutableRel[2];
 
+  /** Creates a SubstitutionVisitor with the default rule set. */
   public SubstitutionVisitor(RelNode target_, RelNode query_) {
+    this(target_, query_, DEFAULT_RULES);
+  }
+
+  /** Creates a SubstitutionVisitor. */
+  public SubstitutionVisitor(RelNode target_, RelNode query_,
+      ImmutableList<UnifyRule> rules) {
     this.cluster = target_.getCluster();
+    this.rules = rules;
     this.query = Holder.of(toMutable(query_));
     this.target = toMutable(target_);
     final Set<MutableRel> parents = Sets.newIdentityHashSet();
@@ -201,28 +217,6 @@ public class SubstitutionVisitor {
     visitor.go(query);
     allNodes.removeAll(parents);
     queryLeaves = ImmutableList.copyOf(allNodes);
-    initUnifyRules();
-    initRuleMap();
-  }
-
-  public void initUnifyRules() {
-    unifyRules =
-            ImmutableList.<UnifyRule>of(
-//          TrivialRule.INSTANCE,
-                    ProjectToProjectUnifyRule.INSTANCE,
-                    FilterToProjectUnifyRule.INSTANCE,
-//          ProjectToFilterUnifyRule.INSTANCE,
-                    FilterToFilterUnifyRule.INSTANCE,
-                    AggregateToAggregateUnifyRule.INSTANCE,
-                    AggregateOnProjectToAggregateUnifyRule.INSTANCE);
-  }
-
-  public void initRuleMap() {
-    this.RULE_MAP.clear();
-  }
-
-  public MutableRel[] getSlots() {
-    return slots;
   }
 
   private static MutableRel toMutable(RelNode rel) {
@@ -619,23 +613,23 @@ public class SubstitutionVisitor {
     return rule.apply(call);
   }
 
-  private static List<UnifyRule> applicableRules(MutableRel query,
+  private List<UnifyRule> applicableRules(MutableRel query,
       MutableRel target) {
     final Class queryClass = query.getClass();
     final Class targetClass = target.getClass();
     final Pair<Class, Class> key = Pair.of(queryClass, targetClass);
-    List<UnifyRule> list = RULE_MAP.get(key);
+    List<UnifyRule> list = ruleMap.get(key);
     if (list == null) {
       final ImmutableList.Builder<UnifyRule> builder =
           ImmutableList.builder();
-      for (UnifyRule rule : unifyRules) {
+      for (UnifyRule rule : rules) {
         //noinspection unchecked
         if (mightMatch(rule, queryClass, targetClass)) {
           builder.add(rule);
         }
       }
       list = builder.build();
-      RULE_MAP.put(key, list);
+      ruleMap.put(key, list);
     }
     return list;
   }
@@ -648,6 +642,7 @@ public class SubstitutionVisitor {
 
   /** Exception thrown to exit a matcher. Not really an error. */
   protected static class MatchFailed extends ControlFlowException {
+    @SuppressWarnings("ThrowableInstanceNeverThrown")
     public static final MatchFailed INSTANCE = new MatchFailed();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5a365609/core/src/main/java/org/apache/calcite/plan/VisitorDataContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/VisitorDataContext.java b/core/src/main/java/org/apache/calcite/plan/VisitorDataContext.java
index a941347..c0fe8a8 100644
--- a/core/src/main/java/org/apache/calcite/plan/VisitorDataContext.java
+++ b/core/src/main/java/org/apache/calcite/plan/VisitorDataContext.java
@@ -67,17 +67,17 @@ public class VisitorDataContext implements DataContext {
       return null;
     }
   }
-  public static DataContext getDataContext(RelNode targetRel, LogicalFilter queryRel) {
-    return getDataContext(targetRel.getRowType(), queryRel.getCondition());
+  public static DataContext of(RelNode targetRel, LogicalFilter queryRel) {
+    return of(targetRel.getRowType(), queryRel.getCondition());
   }
 
-  public static DataContext getDataContext(RelDataType rowType, RexNode rex) {
-    int size = rowType.getFieldList().size();
-    Object[] values = new Object[size];
-    List<RexNode> operands = ((RexCall) rex).getOperands();
+  public static DataContext of(RelDataType rowType, RexNode rex) {
+    final int size = rowType.getFieldList().size();
+    final Object[] values = new Object[size];
+    final List<RexNode> operands = ((RexCall) rex).getOperands();
     final RexNode firstOperand = operands.get(0);
     final RexNode secondOperand = operands.get(1);
-    final Pair<Integer, ? extends Object> value = getValue(firstOperand, secondOperand);
+    final Pair<Integer, ?> value = getValue(firstOperand, secondOperand);
     if (value != null) {
       int index = value.getKey();
       values[index] = value.getValue();
@@ -87,12 +87,12 @@ public class VisitorDataContext implements DataContext {
     }
   }
 
-  public static DataContext getDataContext(RelDataType rowType, List<Pair<RexInputRef,
-      RexNode>> usgList) {
-    int size = rowType.getFieldList().size();
-    Object[] values = new Object[size];
-    for (Pair<RexInputRef, RexNode> elem: usgList) {
-      Pair<Integer, ? extends Object> value = getValue(elem.getKey(), elem.getValue());
+  public static DataContext of(RelDataType rowType,
+      List<Pair<RexInputRef, RexNode>> usageList) {
+    final int size = rowType.getFieldList().size();
+    final Object[] values = new Object[size];
+    for (Pair<RexInputRef, RexNode> elem : usageList) {
+      Pair<Integer, ?> value = getValue(elem.getKey(), elem.getValue());
       if (value == null) {
         return null;
       }
@@ -102,46 +102,40 @@ public class VisitorDataContext implements DataContext {
     return new VisitorDataContext(values);
   }
 
-  public static Pair<Integer, ? extends Object> getValue(RexNode inputRef, RexNode literal) {
+  public static Pair<Integer, ?> getValue(RexNode inputRef, RexNode literal) {
     inputRef = removeCast(inputRef);
     literal = removeCast(literal);
 
     if (inputRef instanceof RexInputRef
         && literal instanceof RexLiteral)  {
-      Integer index = ((RexInputRef) inputRef).getIndex();
+      final int index = ((RexInputRef) inputRef).getIndex();
       Object value = ((RexLiteral) literal).getValue();
       final RelDataType type = inputRef.getType();
 
       switch (type.getSqlTypeName()) {
       case INTEGER:
         if (value instanceof BigDecimal) {
-          final Integer intValue = new Integer(((BigDecimal) value).intValue());
-          return Pair.of(index, intValue);
+          return Pair.of(index, ((BigDecimal) value).intValue());
         }
       case DOUBLE:
         if (value instanceof BigDecimal) {
-          return Pair.of(index,
-              new Double(((BigDecimal) value).doubleValue()));
+          return Pair.of(index, ((BigDecimal) value).doubleValue());
         }
       case REAL:
         if (value instanceof BigDecimal) {
-          return Pair.of(index,
-              new Float(((BigDecimal) value).floatValue()));
+          return Pair.of(index, ((BigDecimal) value).floatValue());
         }
       case BIGINT:
         if (value instanceof BigDecimal) {
-          return Pair.of(index,
-              new Long(((BigDecimal) value).longValue()));
+          return Pair.of(index, ((BigDecimal) value).longValue());
         }
       case SMALLINT:
         if (value instanceof BigDecimal) {
-          return Pair.of(index,
-              new Short(((BigDecimal) value).shortValue()));
+          return Pair.of(index, ((BigDecimal) value).shortValue());
         }
       case TINYINT:
         if (value instanceof BigDecimal) {
-          return Pair.of(index,
-              new Short(((BigDecimal) value).byteValue()));
+          return Pair.of(index, (short) ((BigDecimal) value).byteValue());
         }
       case DECIMAL:
         if (value instanceof BigDecimal) {
@@ -158,13 +152,12 @@ public class VisitorDataContext implements DataContext {
         }
       case CHAR:
         if (value instanceof NlsString) {
-          // TODO: Support coallation. Not supported in {@link #NlsString} compare too.
+          // TODO: Support collation. Not supported in NlsString compare too.
           final NlsString nl = (NlsString) value;
-          Character c = new Character(nl.getValue().charAt(0));
-          return Pair.of(index, c);
+          return Pair.of(index, nl.getValue().charAt(0));
         }
       default:
-        //TODO: Support few more supported cases
+        // TODO: Support few more supported cases
         return Pair.of(index, value);
       }
     }
@@ -184,3 +177,5 @@ public class VisitorDataContext implements DataContext {
     return inputRef;
   }
 }
+
+// End VisitorDataContext.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5a365609/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
index 302a02e..52b5497 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
@@ -113,7 +113,7 @@ public class AggregateFilterTransposeRule extends RelOptRule {
       if (aggregate.indicator) {
         ImmutableList.Builder<ImmutableBitSet> newGroupingSetsBuilder =
                 ImmutableList.builder();
-        for (ImmutableBitSet groupingSet: aggregate.getGroupSets()) {
+        for (ImmutableBitSet groupingSet : aggregate.getGroupSets()) {
           final ImmutableBitSet.Builder newGroupingSet =
                   ImmutableBitSet.builder();
           for (int c : groupingSet) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5a365609/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
index 5f78489..df55f0d 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
@@ -156,7 +156,7 @@ public class AggregateUnionTransposeRule extends RelOptRule {
   private List<AggregateCall> transformAggCalls(RelNode input, int groupCount,
       List<AggregateCall> origCalls) {
     final List<AggregateCall> newCalls = Lists.newArrayList();
-    for (Ord<AggregateCall> ord: Ord.zip(origCalls)) {
+    for (Ord<AggregateCall> ord : Ord.zip(origCalls)) {
       final AggregateCall origCall = ord.e;
       if (origCall.isDistinct()
           || !SUPPORTED_AGGREGATES.containsKey(origCall.getAggregation()

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5a365609/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
index 720e33d..b364ef3 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
@@ -131,7 +131,7 @@ public class FilterAggregateTransposeRule extends RelOptRule {
       // If grouping sets are used, the filter can be pushed if
       // the columns referenced in the predicate are present in
       // all the grouping sets.
-      for (ImmutableBitSet groupingSet: aggregate.getGroupSets()) {
+      for (ImmutableBitSet groupingSet : aggregate.getGroupSets()) {
         if (!groupingSet.contains(rCols)) {
           return false;
         }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5a365609/core/src/main/java/org/apache/calcite/sql/SqlKind.java
----------------------------------------------------------------------
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 244a241..0bdb82c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -728,6 +728,26 @@ public enum SqlKind {
           LESS_THAN, GREATER_THAN,
           GREATER_THAN_OR_EQUAL, LESS_THAN_OR_EQUAL);
 
+  /** Returns the kind that corresponds to this operator but in the opposite
+   * direction. Or returns this, if this kind is not reversible.
+   *
+   * <p>For example, {@code GREATER_THAN.reverse()} returns {@link #LESS_THAN}.
+   */
+  public SqlKind reverse() {
+    switch (this) {
+    case GREATER_THAN:
+      return LESS_THAN;
+    case GREATER_THAN_OR_EQUAL:
+      return LESS_THAN_OR_EQUAL;
+    case LESS_THAN:
+      return GREATER_THAN;
+    case LESS_THAN_OR_EQUAL:
+      return GREATER_THAN_OR_EQUAL;
+    default:
+      return this;
+    }
+  }
+
   /**
    * Returns whether this {@code SqlKind} belongs to a given category.
    *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5a365609/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java b/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
index b56c928..d4780a6 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
@@ -31,6 +31,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.util.NlsString;
 
+import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Assert;
@@ -38,6 +39,7 @@ import org.junit.Test;
 
 import java.math.BigDecimal;
 import java.util.ArrayList;
+import java.util.Calendar;
 import java.util.List;
 
 import static org.hamcrest.CoreMatchers.equalTo;
@@ -70,48 +72,48 @@ public class RexExecutorTest {
   /** Tests an executor that uses variables stored in a {@link DataContext}.
    * Can change the value of the variable and execute again. */
   @Test public void testVariableExecution() throws Exception {
-    check(new Action() {
-      public void check(RexBuilder rexBuilder, RexExecutorImpl executor) {
-        Object[] values = new Object[1];
-        final DataContext testContext = new TestDataContext(values);
-        final RelDataType varchar = rexBuilder.getTypeFactory().createSqlType(
-            SqlTypeName.VARCHAR);
-        final RelDataType integer = rexBuilder.getTypeFactory().createSqlType(
-            SqlTypeName.INTEGER);
-        // calcite is internally creating the creating the input ref via a
-        // RexRangeRef
-        // which eventually leads to a RexInputRef. So we are good.
-        final RexInputRef input = rexBuilder.makeInputRef(varchar, 0);
-        final RexNode lengthArg = rexBuilder.makeLiteral(3, integer, true);
-        final RexNode substr =
-            rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING, input,
-                lengthArg);
-        ImmutableList<RexNode> constExps = ImmutableList.of(substr);
-
-        final RelDataTypeFactory typeFactory = rexBuilder.getTypeFactory();
-        final RelDataType rowType = typeFactory.builder()
-            .add("someStr", varchar)
-            .build();
-
-        final RexExecutable exec = executor.getExecutable(rexBuilder,
-            constExps, rowType);
-        exec.setDataContext(testContext);
-        values[0] = "Hello World";
-        Object[] result = exec.execute();
-        assertTrue(result[0] instanceof String);
-        assertThat((String) result[0], equalTo("llo World"));
-        values[0] = "Calcite";
-        result = exec.execute();
-        assertTrue(result[0] instanceof String);
-        assertThat((String) result[0], equalTo("lcite"));
-      }
-    });
+    check(
+        new Action() {
+          public void check(RexBuilder rexBuilder, RexExecutorImpl executor) {
+            Object[] values = new Object[1];
+            final DataContext testContext = new TestDataContext(values);
+            final RelDataTypeFactory typeFactory = rexBuilder.getTypeFactory();
+            final RelDataType varchar =
+                typeFactory.createSqlType(SqlTypeName.VARCHAR);
+            final RelDataType integer =
+                typeFactory.createSqlType(SqlTypeName.INTEGER);
+            // Calcite is internally creating the input ref via a RexRangeRef
+            // which eventually leads to a RexInputRef. So we are good.
+            final RexInputRef input = rexBuilder.makeInputRef(varchar, 0);
+            final RexNode lengthArg = rexBuilder.makeLiteral(3, integer, true);
+            final RexNode substr =
+                rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING, input,
+                    lengthArg);
+            ImmutableList<RexNode> constExps = ImmutableList.of(substr);
+
+            final RelDataType rowType = typeFactory.builder()
+                .add("someStr", varchar)
+                .build();
+
+            final RexExecutable exec = executor.getExecutable(rexBuilder,
+                constExps, rowType);
+            exec.setDataContext(testContext);
+            values[0] = "Hello World";
+            Object[] result = exec.execute();
+            assertTrue(result[0] instanceof String);
+            assertThat((String) result[0], equalTo("llo World"));
+            values[0] = "Calcite";
+            result = exec.execute();
+            assertTrue(result[0] instanceof String);
+            assertThat((String) result[0], equalTo("lcite"));
+          }
+        });
   }
 
   @Test public void testConstant() throws Exception {
     check(new Action() {
       public void check(RexBuilder rexBuilder, RexExecutorImpl executor) {
-        final List<RexNode> reducedValues = new ArrayList<RexNode>();
+        final List<RexNode> reducedValues = new ArrayList<>();
         final RexLiteral ten = rexBuilder.makeExactLiteral(BigDecimal.TEN);
         executor.reduce(rexBuilder, ImmutableList.<RexNode>of(ten),
             reducedValues);
@@ -123,10 +125,68 @@ public class RexExecutorTest {
     });
   }
 
+  /** Reduces several expressions to constants. */
+  @Test public void testConstant2() throws Exception {
+    // Same as testConstant; 10 -> 10
+    checkConstant(10L,
+        new Function<RexBuilder, RexNode>() {
+          public RexNode apply(RexBuilder rexBuilder) {
+            return rexBuilder.makeExactLiteral(BigDecimal.TEN);
+          }
+        });
+    // 10 + 1 -> 11
+    checkConstant(11L,
+        new Function<RexBuilder, RexNode>() {
+          public RexNode apply(RexBuilder rexBuilder) {
+            return rexBuilder.makeCall(SqlStdOperatorTable.PLUS,
+                rexBuilder.makeExactLiteral(BigDecimal.TEN),
+                rexBuilder.makeExactLiteral(BigDecimal.ONE));
+          }
+        });
+    // date 'today' <= date 'today' -> true
+    checkConstant(true,
+        new Function<RexBuilder, RexNode>() {
+          public RexNode apply(RexBuilder rexBuilder) {
+            Calendar calendar = Calendar.getInstance();
+            return rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
+                rexBuilder.makeDateLiteral(calendar),
+                rexBuilder.makeDateLiteral(calendar));
+          }
+        });
+    // date 'today' < date 'today' -> false
+    checkConstant(false,
+        new Function<RexBuilder, RexNode>() {
+          public RexNode apply(RexBuilder rexBuilder) {
+            Calendar calendar = Calendar.getInstance();
+            return rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN,
+                rexBuilder.makeDateLiteral(calendar),
+                rexBuilder.makeDateLiteral(calendar));
+          }
+        });
+  }
+
+  private void checkConstant(final Object operand,
+      final Function<RexBuilder, RexNode> function) throws Exception {
+    check(
+        new Action() {
+          public void check(RexBuilder rexBuilder, RexExecutorImpl executor) {
+            final List<RexNode> reducedValues = new ArrayList<>();
+            final RexNode expression = function.apply(rexBuilder);
+            assert expression != null;
+            executor.reduce(rexBuilder, ImmutableList.of(expression),
+                reducedValues);
+            assertThat(reducedValues.size(), equalTo(1));
+            assertThat(reducedValues.get(0), instanceOf(RexLiteral.class));
+            assertThat(((RexLiteral) reducedValues.get(0)).getValue2(),
+                equalTo(operand));
+          }
+        });
+  }
+
   @Test public void testSubstring() throws Exception {
     check(new Action() {
       public void check(RexBuilder rexBuilder, RexExecutorImpl executor) {
-        final List<RexNode> reducedValues = new ArrayList<RexNode>();
+        final List<RexNode> reducedValues = new ArrayList<>();
         final RexLiteral hello =
             rexBuilder.makeCharLiteral(
                 new NlsString("Hello world!", null, null));

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5a365609/core/src/test/java/org/apache/calcite/test/DiffRepository.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/DiffRepository.java b/core/src/test/java/org/apache/calcite/test/DiffRepository.java
index fb6b325..572dde9 100644
--- a/core/src/test/java/org/apache/calcite/test/DiffRepository.java
+++ b/core/src/test/java/org/apache/calcite/test/DiffRepository.java
@@ -435,7 +435,6 @@ public class DiffRepository {
             tag,
             expected2Canonical,
             actualCanonical);
-        amend(expected, actual);
       } catch (ComparisonFailure e) {
         amend(expected, actual);
         throw e;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5a365609/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 7e693b0..720798e 100644
--- a/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
@@ -27,6 +27,7 @@ import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexExecutorImpl;
 import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Schemas;
@@ -34,9 +35,9 @@ import org.apache.calcite.server.CalciteServerStatement;
 import org.apache.calcite.sql.SqlCollation;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.Holder;
 import org.apache.calcite.util.NlsString;
 
-import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -50,399 +51,320 @@ import java.sql.Timestamp;
 import java.util.Calendar;
 
 /**
- * Tests the RexImplication checker
+ * Unit tests for {@link RexImplicationChecker}.
  */
 public class RexImplicationCheckerTest {
   //~ Instance fields --------------------------------------------------------
 
-  private RexBuilder rexBuilder = null;
-  private RexNode bl;
-  private RexNode i;
-  private RexNode dec;
-  private RexNode lg;
-  private RexNode sh;
-  private RexNode by;
-  private RexNode fl;
-  private RexNode dt;
-  private RexNode ch;
-  private RexNode ts;
-  private RexNode t;
-
-  private RelDataType boolRelDataType;
-  private RelDataType intRelDataType;
-  private RelDataType decRelDataType;
-  private RelDataType longRelDataType;
-  private RelDataType shortDataType;
-  private RelDataType byteDataType;
-  private RelDataType floatDataType;
-  private RelDataType charDataType;
-  private RelDataType dateDataType;
-  private RelDataType timeStampDataType;
-  private RelDataType timeDataType;
-  private RelDataTypeFactory typeFactory;
-  private RexImplicationChecker checker;
-  private RelDataType rowType;
-  private RexExecutorImpl executor;
-
   //~ Methods ----------------------------------------------------------------
 
-  @Before
-  public void setUp() {
-    typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
-    rexBuilder = new RexBuilder(typeFactory);
-    boolRelDataType = typeFactory.createJavaType(Boolean.class);
-    intRelDataType = typeFactory.createJavaType(Integer.class);
-    decRelDataType = typeFactory.createJavaType(Double.class);
-    longRelDataType = typeFactory.createJavaType(Long.class);
-    shortDataType = typeFactory.createJavaType(Short.class);
-    byteDataType = typeFactory.createJavaType(Byte.class);
-    floatDataType = typeFactory.createJavaType(Float.class);
-    charDataType = typeFactory.createJavaType(Character.class);
-    dateDataType = typeFactory.createJavaType(Date.class);
-    timeStampDataType = typeFactory.createJavaType(Timestamp.class);
-    timeDataType = typeFactory.createJavaType(Time.class);
-
-    bl = new RexInputRef(
-        0,
-        typeFactory.createTypeWithNullability(boolRelDataType, true));
-    i = new RexInputRef(
-        1,
-        typeFactory.createTypeWithNullability(intRelDataType, true));
-    dec = new RexInputRef(
-        2,
-        typeFactory.createTypeWithNullability(decRelDataType, true));
-    lg = new RexInputRef(
-        3,
-        typeFactory.createTypeWithNullability(longRelDataType, true));
-    sh = new RexInputRef(
-        4,
-        typeFactory.createTypeWithNullability(shortDataType, true));
-    by = new RexInputRef(
-        5,
-        typeFactory.createTypeWithNullability(byteDataType, true));
-    fl = new RexInputRef(
-        6,
-        typeFactory.createTypeWithNullability(floatDataType, true));
-    ch = new RexInputRef(
-        7,
-        typeFactory.createTypeWithNullability(charDataType, true));
-    dt = new RexInputRef(
-        8,
-        typeFactory.createTypeWithNullability(dateDataType, true));
-    ts = new RexInputRef(
-        9,
-        typeFactory.createTypeWithNullability(timeStampDataType, true));
-    t = new RexInputRef(
-        10,
-        typeFactory.createTypeWithNullability(timeDataType, true));
-
-    rowType =  typeFactory.builder()
-        .add("bool", boolRelDataType)
-        .add("int", intRelDataType)
-        .add("dec", decRelDataType)
-        .add("long", longRelDataType)
-        .add("short", shortDataType)
-        .add("byte", byteDataType)
-        .add("float", floatDataType)
-        .add("char", charDataType)
-        .add("date", dateDataType)
-        .add("timestamp", timeStampDataType)
-        .add("time", timeDataType)
-        .build();
-
-    Frameworks.withPrepare(
-        new Frameworks.PrepareAction<Void>() {
-          public Void apply(RelOptCluster cluster,
-                            RelOptSchema relOptSchema,
-                            SchemaPlus rootSchema,
-                            CalciteServerStatement statement) {
-            DataContext dataContext =
-                Schemas.createDataContext(statement.getConnection());
-            executor = new RexExecutorImpl(dataContext);
-            return null;
-          }
-        });
-
-    checker = new RexImplicationChecker(rexBuilder, executor, rowType);
-  }
-
-  private void checkImplies(RexNode node1, RexNode node2) {
-    assertTrue(node1.toString() + " doesnot imply " + node2.toString()
-        + " when it should.", checker.implies(node1, node2));
-  }
-
-  private void checkNotImplies(RexNode node1, RexNode node2) {
-    assertFalse(node1.toString() + " implies " + node2.toString()
-        + " when it should not", checker.implies(node1, node2));
-  }
-
   // Simple Tests for Operators
   @Test public void testSimpleGreaterCond() {
-    RexNode node1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(10)));
-
-    RexNode node2 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(30)));
-
-    RexNode node3 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(30)));
-
-    RexNode node4 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(10)));
-
-    RexNode node5 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.EQUALS,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(30)));
-
-    RexNode node6 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.NOT_EQUALS,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(10)));
-
-    checkImplies(node2, node1);
-    checkNotImplies(node1, node2);
-    checkNotImplies(node1, node3);
-    checkImplies(node3, node1);
-    checkImplies(node5, node1);
-    checkNotImplies(node1, node5);
-    checkNotImplies(node1, node6);
-    checkNotImplies(node4, node6);
+    final Fixture f = new Fixture();
+    final RexNode node1 = f.gt(f.i, f.literal(10));
+    final RexNode node2 = f.gt(f.i, f.literal(30));
+    final RexNode node3 = f.ge(f.i, f.literal(30));
+    final RexNode node4 = f.ge(f.i, f.literal(10));
+    final RexNode node5 = f.eq(f.i, f.literal(30));
+    final RexNode node6 = f.ne(f.i, f.literal(10));
+
+    f.checkImplies(node2, node1);
+    f.checkNotImplies(node1, node2);
+    f.checkNotImplies(node1, node3);
+    f.checkImplies(node3, node1);
+    f.checkImplies(node5, node1);
+    f.checkNotImplies(node1, node5);
+    f.checkNotImplies(node1, node6);
+    f.checkNotImplies(node4, node6);
     // TODO: Need to support Identity
-    //checkImplies(node1, node1);
-    //checkImplies(node3, node3);
+    //f.checkImplies(node1, node1);
+    //f.checkImplies(node3, node3);
   }
 
   @Test public void testSimpleLesserCond() {
-    RexNode node1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(10)));
-
-    RexNode node2 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(30)));
-
-    RexNode node3 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(30)));
-
-    RexNode node4 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(10)));
-
-    RexNode node5 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.EQUALS,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(10)));
-
-    RexNode node6 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.NOT_EQUALS,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(10)));
-
-    checkImplies(node1, node2);
-    checkNotImplies(node2, node1);
-    checkImplies(node1, node3);
-    checkNotImplies(node3, node1);
-    checkImplies(node5, node2);
-    checkNotImplies(node2, node5);
-    checkNotImplies(node1, node5);
-    checkNotImplies(node1, node6);
-    checkNotImplies(node4, node6);
+    final Fixture f = new Fixture();
+    final RexNode node1 = f.lt(f.i, f.literal(10));
+    final RexNode node2 = f.lt(f.i, f.literal(30));
+    final RexNode node3 = f.le(f.i, f.literal(30));
+    final RexNode node4 = f.le(f.i, f.literal(10));
+    final RexNode node5 = f.eq(f.i, f.literal(10));
+    final RexNode node6 = f.ne(f.i, f.literal(10));
+
+    f.checkImplies(node1, node2);
+    f.checkNotImplies(node2, node1);
+    f.checkImplies(node1, node3);
+    f.checkNotImplies(node3, node1);
+    f.checkImplies(node5, node2);
+    f.checkNotImplies(node2, node5);
+    f.checkNotImplies(node1, node5);
+    f.checkNotImplies(node1, node6);
+    f.checkNotImplies(node4, node6);
     // TODO: Need to support Identity
-    //checkImplies(node1, node1);
-    //checkImplies(node3, node3);
+    //f.checkImplies(node1, node1);
+    //f.checkImplies(node3, node3);
   }
 
   @Test public void testSimpleEq() {
-    RexNode node1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.EQUALS,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(30)));
-
-    RexNode node2 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.NOT_EQUALS,
-            i,
-            rexBuilder.makeExactLiteral(new BigDecimal(10)));
+    final Fixture f = new Fixture();
+    final RexNode node1 = f.eq(f.i, f.literal(30));
+    final RexNode node2 = f.ne(f.i, f.literal(10));
 
     //Check Identity
-    checkImplies(node1, node1);
+    f.checkImplies(node1, node1);
     //TODO: Support Identity
-    // checkImplies(node2, node2);
-    checkImplies(node1, node2);
-    checkNotImplies(node2, node1);
+    // f.checkImplies(node2, node2);
+    f.checkImplies(node1, node2);
+    f.checkNotImplies(node2, node1);
   }
 
   // Simple Tests for DataTypes
   @Test public void testSimpleDec() {
-    RexNode node1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN,
-            dec,
-            rexBuilder.makeApproxLiteral(new BigDecimal(30.9)));
-
-    RexNode node2 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN,
-            dec,
-            rexBuilder.makeApproxLiteral(new BigDecimal(40.33)));
-
-    checkImplies(node1, node2);
-    checkNotImplies(node2, node1);
+    final Fixture f = new Fixture();
+    final RexNode node1 = f.lt(f.dec, f.floatLiteral(30.9));
+    final RexNode node2 = f.lt(f.dec, f.floatLiteral(40.33));
+
+    f.checkImplies(node1, node2);
+    f.checkNotImplies(node2, node1);
   }
 
   @Test public void testSimpleBoolean() {
-    RexNode node1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.EQUALS,
-            bl,
-            rexBuilder.makeLiteral(true));
-
-    RexNode node2 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.EQUALS,
-            bl,
-            rexBuilder.makeLiteral(false));
+    final Fixture f = new Fixture();
+    final RexNode node1 = f.eq(f.bl, f.rexBuilder.makeLiteral(true));
+    final RexNode node2 = f.eq(f.bl, f.rexBuilder.makeLiteral(false));
 
     //TODO: Need to support false => true
-    //checkImplies(node2, node1);
-    checkNotImplies(node1, node2);
+    //f.checkImplies(node2, node1);
+    f.checkNotImplies(node1, node2);
   }
 
   @Test public void testSimpleLong() {
-    RexNode node1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
-            lg,
-            rexBuilder.makeLiteral(new Long(324324L), longRelDataType, true));
-
-    RexNode node2 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN,
-            lg,
-            rexBuilder.makeLiteral(new Long(324325L), longRelDataType, true));
-
-    checkImplies(node2, node1);
-    checkNotImplies(node1, node2);
+    final Fixture f = new Fixture();
+    final RexNode node1 = f.ge(f.lg, f.longLiteral(324324L));
+    final RexNode node2 = f.gt(f.lg, f.longLiteral(324325L));
+
+    f.checkImplies(node2, node1);
+    f.checkNotImplies(node1, node2);
   }
 
   @Test public void testSimpleShort() {
-    RexNode node1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
-            sh,
-            rexBuilder.makeLiteral(new Short((short) 10), shortDataType, true));
-
-    RexNode node2 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
-            sh,
-            rexBuilder.makeLiteral(new Short((short) 11), shortDataType, true));
-
-    checkImplies(node2, node1);
-    checkNotImplies(node1, node2);
+    final Fixture f = new Fixture();
+    final RexNode node1 = f.ge(f.sh, f.shortLiteral((short) 10));
+    final RexNode node2 = f.ge(f.sh, f.shortLiteral((short) 11));
+
+    f.checkImplies(node2, node1);
+    f.checkNotImplies(node1, node2);
   }
 
   @Test public void testSimpleChar() {
-    RexNode node1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
-            ch,
-            rexBuilder.makeCharLiteral(new NlsString("b", null, SqlCollation.COERCIBLE)));
-
-    RexNode node2 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
-            ch,
-            rexBuilder.makeCharLiteral(new NlsString("a", null, SqlCollation.COERCIBLE)));
-
-    checkImplies(node1, node2);
-    checkNotImplies(node2, node1);
+    final Fixture f = new Fixture();
+    final RexNode node1 = f.ge(f.ch, f.charLiteral("b"));
+    final RexNode node2 = f.ge(f.ch, f.charLiteral("a"));
+
+    f.checkImplies(node1, node2);
+    f.checkNotImplies(node2, node1);
   }
 
+  @Ignore("work in progress")
   @Test public void testSimpleDate() {
-    RexNode node1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
-            dt,
-            rexBuilder.makeDateLiteral(Calendar.getInstance()));
-
-    RexNode node2 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.EQUALS,
-            dt,
-            rexBuilder.makeDateLiteral(Calendar.getInstance()));
-
-    checkImplies(node2, node1);
-    checkNotImplies(node1, node2);
+    final Fixture f = new Fixture();
+    final Calendar instance = Calendar.getInstance();
+    final RexNode node1 = f.ge(f.dt, f.rexBuilder.makeDateLiteral(instance));
+    final RexNode node2 = f.eq(f.dt, f.rexBuilder.makeDateLiteral(instance));
+
+    f.checkImplies(node2, node1);
+    f.checkNotImplies(node1, node2);
   }
 
   @Ignore("work in progress")
   @Test public void testSimpleTimeStamp() {
-    RexNode node1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-            ts,
-            rexBuilder.makeTimestampLiteral(Calendar.getInstance(),
-                timeStampDataType.getPrecision()));
-
-
-    RexNode node2 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-            ts,
-            rexBuilder.makeTimestampLiteral(Calendar.getInstance(),
-                timeStampDataType.getPrecision()));
-
-    checkImplies(node1, node2);
-    checkNotImplies(node2, node1);
+    final Fixture f = new Fixture();
+    final Calendar calendar = Calendar.getInstance();
+    final RexNode node1 = f.le(f.ts, f.timestampLiteral(calendar));
+    final RexNode node2 = f.le(f.ts, f.timestampLiteral(calendar));
+
+    f.checkImplies(node1, node2);
+    f.checkNotImplies(node2, node1);
   }
 
   @Ignore("work in progress")
   @Test public void testSimpleTime() {
-    RexNode node1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-            t,
-            rexBuilder.makeTimeLiteral(Calendar.getInstance(),
-                timeDataType.getPrecision()));
-
-
-    RexNode node2 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-            t,
-            rexBuilder.makeTimestampLiteral(Calendar.getInstance(),
-                timeDataType.getPrecision()));
-
-    checkImplies(node1, node2);
-    checkNotImplies(node2, node1);
+    final Fixture f = new Fixture();
+    final Calendar calendar = Calendar.getInstance();
+    final RexNode node1 = f.le(f.ts, f.timeLiteral(calendar));
+    final RexNode node2 = f.le(f.ts, f.timeLiteral(calendar));
+
+    f.checkImplies(node1, node2);
+    f.checkNotImplies(node2, node1);
   }
 
+  /** Contains all the nourishment a test case could possibly need.
+   *
+   * <p>We put the data in here, rather than as fields in the test case, so that
+   * the data can be garbage-collected as soon as the test has executed.
+   */
+  private static class Fixture {
+    private final RexBuilder rexBuilder;
+    private final RexNode bl;
+    private final RexNode i;
+    private final RexNode dec;
+    private final RexNode lg;
+    private final RexNode sh;
+    private final RexNode by;
+    private final RexNode fl;
+    private final RexNode dt;
+    private final RexNode ch;
+    private final RexNode ts;
+    private final RexNode t;
+
+    private final RelDataType boolRelDataType;
+    private final RelDataType intRelDataType;
+    private final RelDataType decRelDataType;
+    private final RelDataType longRelDataType;
+    private final RelDataType shortDataType;
+    private final RelDataType byteDataType;
+    private final RelDataType floatDataType;
+    private final RelDataType charDataType;
+    private final RelDataType dateDataType;
+    private final RelDataType timeStampDataType;
+    private final RelDataType timeDataType;
+    private final RelDataTypeFactory typeFactory;
+    private final RexImplicationChecker checker;
+    private final RelDataType rowType;
+    private final RexExecutorImpl executor;
+
+    public Fixture() {
+      typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+      rexBuilder = new RexBuilder(typeFactory);
+      boolRelDataType = typeFactory.createJavaType(Boolean.class);
+      intRelDataType = typeFactory.createJavaType(Integer.class);
+      decRelDataType = typeFactory.createJavaType(Double.class);
+      longRelDataType = typeFactory.createJavaType(Long.class);
+      shortDataType = typeFactory.createJavaType(Short.class);
+      byteDataType = typeFactory.createJavaType(Byte.class);
+      floatDataType = typeFactory.createJavaType(Float.class);
+      charDataType = typeFactory.createJavaType(Character.class);
+      dateDataType = typeFactory.createJavaType(Date.class);
+      timeStampDataType = typeFactory.createJavaType(Timestamp.class);
+      timeDataType = typeFactory.createJavaType(Time.class);
+
+      bl = ref(0, this.boolRelDataType);
+      i = ref(1, intRelDataType);
+      dec = ref(2, decRelDataType);
+      lg = ref(3, longRelDataType);
+      sh = ref(4, shortDataType);
+      by = ref(5, byteDataType);
+      fl = ref(6, floatDataType);
+      ch = ref(7, charDataType);
+      dt = ref(8, dateDataType);
+      ts = ref(9, timeStampDataType);
+      t = ref(10, timeDataType);
+
+      rowType = typeFactory.builder()
+          .add("bool", this.boolRelDataType)
+          .add("int", intRelDataType)
+          .add("dec", decRelDataType)
+          .add("long", longRelDataType)
+          .add("short", shortDataType)
+          .add("byte", byteDataType)
+          .add("float", floatDataType)
+          .add("char", charDataType)
+          .add("date", dateDataType)
+          .add("timestamp", timeStampDataType)
+          .add("time", timeDataType)
+          .build();
+
+      final Holder<RexExecutorImpl> holder = Holder.of(null);
+      Frameworks.withPrepare(
+          new Frameworks.PrepareAction<Void>() {
+            public Void apply(RelOptCluster cluster,
+                RelOptSchema relOptSchema,
+                SchemaPlus rootSchema,
+                CalciteServerStatement statement) {
+              DataContext dataContext =
+                  Schemas.createDataContext(statement.getConnection());
+              holder.set(new RexExecutorImpl(dataContext));
+              return null;
+            }
+          });
+
+      executor = holder.get();
+      checker = new RexImplicationChecker(rexBuilder, executor, rowType);
+    }
+
+    RexInputRef ref(int i, RelDataType type) {
+      return new RexInputRef(i,
+          typeFactory.createTypeWithNullability(type, true));
+    }
+
+    RexLiteral literal(int i) {
+      return rexBuilder.makeExactLiteral(new BigDecimal(i));
+    }
+
+    RexNode gt(RexNode node1, RexNode node2) {
+      return rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, node1, node2);
+    }
+
+    RexNode ge(RexNode node1, RexNode node2) {
+      return rexBuilder.makeCall(
+          SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, node1, node2);
+    }
+
+    RexNode eq(RexNode node1, RexNode node2) {
+      return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, node1, node2);
+    }
+
+    RexNode ne(RexNode node1, RexNode node2) {
+      return rexBuilder.makeCall(SqlStdOperatorTable.NOT_EQUALS, node1, node2);
+    }
+
+    RexNode lt(RexNode node1, RexNode node2) {
+      return rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, node1, node2);
+    }
+
+    RexNode le(RexNode node1, RexNode node2) {
+      return rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, node1,
+          node2);
+    }
+
+    RexNode longLiteral(long value) {
+      return rexBuilder.makeLiteral(value, longRelDataType, true);
+    }
+
+    RexNode shortLiteral(short value) {
+      return rexBuilder.makeLiteral(value, shortDataType, true);
+    }
+
+    RexLiteral floatLiteral(double value) {
+      return rexBuilder.makeApproxLiteral(new BigDecimal(value));
+    }
+
+    RexLiteral charLiteral(String z) {
+      return rexBuilder.makeCharLiteral(
+          new NlsString(z, null, SqlCollation.COERCIBLE));
+    }
+
+    RexNode timestampLiteral(Calendar calendar) {
+      return rexBuilder.makeTimestampLiteral(
+          calendar, timeStampDataType.getPrecision());
+    }
+
+    RexNode timeLiteral(Calendar calendar) {
+      return rexBuilder.makeTimestampLiteral(
+          calendar, timeDataType.getPrecision());
+    }
+
+    void checkImplies(RexNode node1, RexNode node2) {
+      final String message =
+          node1 + " does not imply " + node2 + " when it should";
+      assertTrue(message, checker.implies(node1, node2));
+    }
+
+    void checkNotImplies(RexNode node1, RexNode node2) {
+      final String message =
+          node1 + " does implies " + node2 + " when it should not";
+      assertFalse(message, checker.implies(node1, node2));
+    }
+  }
 }
+
+// End RexImplicationCheckerTest.java