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 2016/11/20 22:17:49 UTC

[2/3] calcite git commit: [CALCITE-1494] Infer IS NOT NULL, and project predicates

[CALCITE-1494] Infer IS NOT NULL, and project predicates


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

Branch: refs/heads/master
Commit: 8b1f22ddf7ef3feccdec986402292972e4854980
Parents: 3ccfb9c
Author: Julian Hyde <jh...@apache.org>
Authored: Sun Nov 20 12:48:39 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Nov 20 12:59:37 2016 -0800

----------------------------------------------------------------------
 .../calcite/plan/RexImplicationChecker.java     | 138 +++++++-----
 .../calcite/rel/metadata/RelMdPredicates.java   |  59 ++++-
 .../apache/calcite/test/RelMetadataTest.java    |  71 +++---
 .../calcite/test/RexImplicationCheckerTest.java | 223 +++++++++++--------
 4 files changed, 323 insertions(+), 168 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/8b1f22dd/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 27dbd34..056bffe 100644
--- a/core/src/main/java/org/apache/calcite/plan/RexImplicationChecker.java
+++ b/core/src/main/java/org/apache/calcite/plan/RexImplicationChecker.java
@@ -52,7 +52,9 @@ import java.util.Set;
  * <p>For example:
  * <ul>
  * <li>(x &gt; 10) &rArr; (x &gt; 5)
- * <li>(y = 10) &rArr; (y &lt; 30 OR x &gt; 30)
+ * <li>(x = 10) &rArr; (x &lt; 30 OR y &gt; 30)
+ * <li>(x = 10) &rArr; (x IS NOT NULL)
+ * <li>(x &gt; 10 AND y = 20) &rArr; (x &gt; 5)
  * </ul>
  */
 public class RexImplicationChecker {
@@ -92,9 +94,6 @@ public class RexImplicationChecker {
 
     LOGGER.debug("Checking if {} => {}", first.toString(), second.toString());
 
-    RexCall firstCond = (RexCall) first;
-    RexCall secondCond = (RexCall) second;
-
     // Get DNF
     RexNode firstDnf = RexUtil.toDnf(builder, first);
     RexNode secondDnf = RexUtil.toDnf(builder, second);
@@ -105,45 +104,26 @@ public class RexImplicationChecker {
       return true;
     }
 
-    /** Decomposes DNF into List of Conjunctions.
-     *
-     * <p>For example,
-     * {@code x > 10 AND y > 30) OR (z > 90)}
-     * will be converted to
-     * list of 2 conditions:
-     *
-     * <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 at least
-        // one of the conjunctions in list secondDnfs
-        boolean implyOneConjunction = false;
-        for (RexNode s : secondDnfs) {
-          if (s.isAlwaysFalse()) { // f cannot imply s
-            continue;
-          }
-
-          if (impliesConjunction(f, s)) {
-            // Satisfies one of the condition, so lets
-            // move to next conjunction in firstDnfs
-            implyOneConjunction = true;
-            break;
-          }
-        }
-
-        // If f could not imply even one conjunction in
-        // secondDnfs, then final implication may be false
-        if (!implyOneConjunction) {
-          LOGGER.debug("{} doesnot imply {}", first, second);
-          return false;
-        }
+    // Decompose DNF into a list of conditions, each of which is a conjunction.
+    // For example,
+    //   (x > 10 AND y > 30) OR (z > 90)
+    // is converted to list of 2 conditions:
+    //   (x > 10 AND y > 30)
+    //   z > 90
+    //
+    // Similarly, decompose CNF into a list of conditions, each of which is a
+    // disjunction.
+    List<RexNode> firsts = RelOptUtil.disjunctions(firstDnf);
+    List<RexNode> seconds = RelOptUtil.disjunctions(secondDnf);
+
+    for (RexNode f : firsts) {
+      // Check if f implies at least
+      // one of the conjunctions in list secondDnfs.
+      // If f could not imply even one conjunction in
+      // secondDnfs, then final implication may be false.
+      if (!impliesAny(f, seconds)) {
+        LOGGER.debug("{} does not imply {}", first, second);
+        return false;
       }
     }
 
@@ -151,13 +131,71 @@ public class RexImplicationChecker {
     return true;
   }
 
-  /** Returns whether first implies second (both are conjunctions). */
+  /** Returns whether the predicate {@code first} implies (&rArr;)
+   * at least one predicate in {@code seconds}. */
+  private boolean impliesAny(RexNode first, List<RexNode> seconds) {
+    for (RexNode second : seconds) {
+      if (impliesConjunction(first, second)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** Returns whether the predicate {@code first} implies {@code second} (both
+   * may be conjunctions). */
   private boolean impliesConjunction(RexNode first, RexNode second) {
+    if (implies2(first, second)) {
+      return true;
+    }
+    switch (first.getKind()) {
+    case AND:
+      for (RexNode f : RelOptUtil.conjunctions(first)) {
+        if (implies2(f, second)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  /** Returns whether the predicate {@code first} (not a conjunction)
+   * implies {@code second}. */
+  private boolean implies2(RexNode first, RexNode second) {
+    if (second.isAlwaysFalse()) { // f cannot imply s
+      return false;
+    }
+
+    // E.g. "x is null" implies "x is null".
+    if (RexUtil.eq(first, second)) {
+      return true;
+    }
+
+    // Several things imply "IS NOT NULL"
+    switch (second.getKind()) {
+    case IS_NOT_NULL:
+      final RexNode operand = ((RexCall) second).getOperands().get(0);
+      switch (first.getKind()) {
+      case IS_NOT_NULL:
+      case IS_TRUE:
+      case IS_FALSE:
+      case LESS_THAN:
+      case LESS_THAN_OR_EQUAL:
+      case GREATER_THAN:
+      case GREATER_THAN_OR_EQUAL:
+      case EQUALS:
+      case NOT_EQUALS:
+        if (((RexCall) first).getOperands().contains(operand)) {
+          return true;
+        }
+      }
+    }
+
     final InputUsageFinder firstUsageFinder = new InputUsageFinder();
     final InputUsageFinder secondUsageFinder = new InputUsageFinder();
 
-    RexUtil.apply(firstUsageFinder, new ArrayList<RexNode>(), first);
-    RexUtil.apply(secondUsageFinder, new ArrayList<RexNode>(), second);
+    RexUtil.apply(firstUsageFinder, ImmutableList.<RexNode>of(), first);
+    RexUtil.apply(secondUsageFinder, ImmutableList.<RexNode>of(), second);
 
     // Check Support
     if (!checkSupport(firstUsageFinder, secondUsageFinder)) {
@@ -183,7 +221,7 @@ public class RexImplicationChecker {
     final Set<List<Pair<RexInputRef, RexNode>>> usages =
         Sets.cartesianProduct(usagesBuilder.build());
 
-    for (List usageList : usages) {
+    for (List<Pair<RexInputRef, RexNode>> usageList : usages) {
       // Get the literals from first conjunction and executes second conjunction
       // using them.
       //
@@ -390,10 +428,10 @@ public class RexImplicationChecker {
    * </ul>
    */
   private static class InputUsageFinder extends RexVisitorImpl<Void> {
-    public final Map<RexInputRef, InputRefUsage<SqlOperator, RexNode>>
-    usageMap = new HashMap<>();
+    final Map<RexInputRef, InputRefUsage<SqlOperator, RexNode>> usageMap =
+        new HashMap<>();
 
-    public InputUsageFinder() {
+    InputUsageFinder() {
       super(true);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/8b1f22dd/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
index 0a4e89c..df7e679 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
@@ -194,10 +194,10 @@ public class RelMdPredicates
     // 'columnsMapped' construct a new predicate based on mapping.
     final ImmutableBitSet columnsMapped = columnsMappedBuilder.build();
     for (RexNode r : inputInfo.pulledUpPredicates) {
-      ImmutableBitSet rCols = RelOptUtil.InputFinder.bits(r);
-      if (columnsMapped.contains(rCols)) {
-        r = r.accept(new RexPermuteInputsShuttle(m, input));
-        projectPullUpPredicates.add(r);
+      RexNode r2 = projectPredicate(rexBuilder, r, columnsMapped);
+      if (!r2.isAlwaysTrue()) {
+        r2 = r2.accept(new RexPermuteInputsShuttle(m, input));
+        projectPullUpPredicates.add(r2);
       }
     }
 
@@ -220,6 +220,57 @@ public class RelMdPredicates
     return RelOptPredicateList.of(rexBuilder, projectPullUpPredicates);
   }
 
+  /** Converts a predicate on a particular set of columns into a predicate on
+   * a subset of those columns, weakening if necessary.
+   *
+   * <p>If not possible to simplify, returns {@code true}, which is the weakest
+   * possible predicate.
+   *
+   * <p>Examples:<ol>
+   * <li>The predicate {@code $7 = $9} on columns [7]
+   *     becomes {@code $7 is not null}
+   * <li>The predicate {@code $7 = $9 and $9 = 5} on columns [7] becomes
+   *   {@code $7 = 5}
+   * <li>The predicate
+   *   {@code $7 = $9 and ($9 = $1 or $9 = $2) and $1 > 3 and $2 > 10}
+   *   on columns [7] becomes {@code $7 > 3}
+   * </ol>
+   *
+   * <p>We currently only handle example 1.
+   *
+   * @param r Predicate expression
+   * @param columnsMapped Columns which the final predicate can reference
+   * @return Predicate expression narrowed to reference only certain columns
+   */
+  private RexNode projectPredicate(RexBuilder rexBuilder, RexNode r,
+      ImmutableBitSet columnsMapped) {
+    ImmutableBitSet rCols = RelOptUtil.InputFinder.bits(r);
+    if (columnsMapped.contains(rCols)) {
+      // All required columns are present. No need to weaken.
+      return r;
+    }
+    if (columnsMapped.intersects(rCols)) {
+      final ImmutableBitSet cols2 = columnsMapped.intersect(rCols);
+      switch (r.getKind()) {
+      case EQUALS:
+      case NOT_EQUALS:
+      case GREATER_THAN:
+      case GREATER_THAN_OR_EQUAL:
+      case LESS_THAN:
+      case LESS_THAN_OR_EQUAL:
+        for (RexNode operand : ((RexCall) r).getOperands()) {
+          if (operand instanceof RexInputRef
+              && cols2.get(((RexInputRef) operand).getIndex())) {
+            return rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL,
+                operand);
+          }
+        }
+      }
+    }
+    // Cannot weaken to anything non-trivial
+    return rexBuilder.makeLiteral(true);
+  }
+
   /**
    * Add the Filter condition to the pulledPredicates list from the input.
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/8b1f22dd/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
index 5a12335..b1c2ca5 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -62,7 +62,6 @@ import org.apache.calcite.rel.metadata.RelMdCollation;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
@@ -70,6 +69,7 @@ import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 
@@ -1281,49 +1281,70 @@ public class RelMetadataTest extends SqlToRelTestBase {
 
   private void checkPredicates(RelOptCluster cluster, RelOptTable empTable,
       RelOptTable deptTable) {
-    final RexBuilder rexBuilder = cluster.getRexBuilder();
+    final RelBuilder relBuilder = RelBuilder.proto().create(cluster, null);
     final RelMetadataQuery mq = RelMetadataQuery.instance();
+
     final LogicalTableScan empScan = LogicalTableScan.create(cluster, empTable);
+    relBuilder.push(empScan);
 
     RelOptPredicateList predicates =
         mq.getPulledUpPredicates(empScan);
     assertThat(predicates.pulledUpPredicates.isEmpty(), is(true));
 
-    final LogicalFilter filter =
-        LogicalFilter.create(empScan,
-            rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
-                rexBuilder.makeInputRef(empScan,
-                    empScan.getRowType().getFieldNames().indexOf("EMPNO")),
-                rexBuilder.makeExactLiteral(BigDecimal.ONE)));
+    relBuilder.filter(
+        relBuilder.equals(relBuilder.field("EMPNO"),
+            relBuilder.literal(BigDecimal.ONE)));
 
+    final RelNode filter = relBuilder.peek();
     predicates = mq.getPulledUpPredicates(filter);
     assertThat(predicates.pulledUpPredicates.toString(), is("[=($0, 1)]"));
 
     final LogicalTableScan deptScan =
         LogicalTableScan.create(cluster, deptTable);
+    relBuilder.push(deptScan);
 
-    final RelDataTypeField leftDeptnoField =
-        empScan.getRowType().getFieldList().get(
-            empScan.getRowType().getFieldNames().indexOf("DEPTNO"));
-    final RelDataTypeField rightDeptnoField =
-        deptScan.getRowType().getFieldList().get(
-            deptScan.getRowType().getFieldNames().indexOf("DEPTNO"));
-    final SemiJoin semiJoin =
-        SemiJoin.create(filter, deptScan,
-            rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
-                rexBuilder.makeInputRef(leftDeptnoField.getType(),
-                    leftDeptnoField.getIndex()),
-                rexBuilder.makeInputRef(rightDeptnoField.getType(),
-                    rightDeptnoField.getIndex()
-                        + empScan.getRowType().getFieldCount())),
-            ImmutableIntList.of(leftDeptnoField.getIndex()),
-            ImmutableIntList.of(rightDeptnoField.getIndex()
-                    + empScan.getRowType().getFieldCount()));
+    relBuilder.semiJoin(
+        relBuilder.equals(relBuilder.field(2, 0, "DEPTNO"),
+            relBuilder.field(2, 1, "DEPTNO")));
+    final SemiJoin semiJoin = (SemiJoin) relBuilder.build();
 
     predicates = mq.getPulledUpPredicates(semiJoin);
     assertThat(predicates.pulledUpPredicates, sortsAs("[=($0, 1)]"));
     assertThat(predicates.leftInferredPredicates, sortsAs("[]"));
     assertThat(predicates.rightInferredPredicates.isEmpty(), is(true));
+
+    // Create a Join similar to the previous SemiJoin
+    relBuilder.push(filter);
+    relBuilder.push(deptScan);
+    relBuilder.join(JoinRelType.INNER,
+        relBuilder.equals(relBuilder.field(2, 0, "DEPTNO"),
+            relBuilder.field(2, 1, "DEPTNO")));
+
+    relBuilder.project(relBuilder.field("DEPTNO"));
+    final RelNode project = relBuilder.peek();
+    predicates = mq.getPulledUpPredicates(project);
+    // No inferred predicates, because we already know DEPTNO is NOT NULL
+    assertThat(predicates.pulledUpPredicates, sortsAs("[]"));
+    assertThat(project.getRowType().getFullTypeString(),
+        is("RecordType(INTEGER NOT NULL DEPTNO) NOT NULL"));
+    assertThat(predicates.leftInferredPredicates.isEmpty(), is(true));
+    assertThat(predicates.rightInferredPredicates.isEmpty(), is(true));
+
+    // Create a Join similar to the previous Join, but joining on MGR, which
+    // is nullable. From the join condition "e.MGR = d.DEPTNO" we can deduce
+    // the projected predicate "IS NOT NULL($0)".
+    relBuilder.push(filter);
+    relBuilder.push(deptScan);
+    relBuilder.join(JoinRelType.INNER,
+        relBuilder.equals(relBuilder.field(2, 0, "MGR"),
+            relBuilder.field(2, 1, "DEPTNO")));
+
+    relBuilder.project(relBuilder.field("MGR"));
+    final RelNode project2 = relBuilder.peek();
+    predicates = mq.getPulledUpPredicates(project2);
+    assertThat(predicates.pulledUpPredicates, sortsAs("[IS NOT NULL($0)]"));
+    assertThat(predicates.leftInferredPredicates.isEmpty(), is(true));
+    assertThat(predicates.rightInferredPredicates.isEmpty(), is(true));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/8b1f22dd/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 a54167d..4464019 100644
--- a/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
@@ -61,60 +61,60 @@ public class RexImplicationCheckerTest {
   // Simple Tests for Operators
   @Test public void testSimpleGreaterCond() {
     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
-    //f.checkImplies(node1, node1);
-    //f.checkImplies(node3, node3);
+    final RexNode iGt10 = f.gt(f.i, f.literal(10));
+    final RexNode iGt30 = f.gt(f.i, f.literal(30));
+    final RexNode iGe30 = f.ge(f.i, f.literal(30));
+    final RexNode iGe10 = f.ge(f.i, f.literal(10));
+    final RexNode iEq30 = f.eq(f.i, f.literal(30));
+    final RexNode iNe10 = f.ne(f.i, f.literal(10));
+
+    f.checkImplies(iGt30, iGt10);
+    f.checkNotImplies(iGt10, iGt30);
+    f.checkNotImplies(iGt10, iGe30);
+    f.checkImplies(iGe30, iGt10);
+    f.checkImplies(iEq30, iGt10);
+    f.checkNotImplies(iGt10, iEq30);
+    f.checkNotImplies(iGt10, iNe10);
+    f.checkNotImplies(iGe10, iNe10);
+    // identity
+    f.checkImplies(iGt10, iGt10);
+    f.checkImplies(iGe30, iGe30);
   }
 
   @Test public void testSimpleLesserCond() {
     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
-    //f.checkImplies(node1, node1);
-    //f.checkImplies(node3, node3);
+    final RexNode iLt10 = f.lt(f.i, f.literal(10));
+    final RexNode iLt30 = f.lt(f.i, f.literal(30));
+    final RexNode iLe30 = f.le(f.i, f.literal(30));
+    final RexNode iLe10 = f.le(f.i, f.literal(10));
+    final RexNode iEq10 = f.eq(f.i, f.literal(10));
+    final RexNode iNe10 = f.ne(f.i, f.literal(10));
+
+    f.checkImplies(iLt10, iLt30);
+    f.checkNotImplies(iLt30, iLt10);
+    f.checkImplies(iLt10, iLe30);
+    f.checkNotImplies(iLe30, iLt10);
+    f.checkImplies(iEq10, iLt30);
+    f.checkNotImplies(iLt30, iEq10);
+    f.checkNotImplies(iLt10, iEq10);
+    f.checkNotImplies(iLt10, iNe10);
+    f.checkNotImplies(iLe10, iNe10);
+    // identity
+    f.checkImplies(iLt10, iLt10);
+    f.checkImplies(iLe30, iLe30);
   }
 
   @Test public void testSimpleEq() {
     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
-    f.checkImplies(node1, node1);
-    //TODO: Support Identity
-    // f.checkImplies(node2, node2);
-    f.checkImplies(node1, node2);
-    f.checkNotImplies(node2, node1);
+    final RexNode iEq30 = f.eq(f.i, f.literal(30));
+    final RexNode iNe10 = f.ne(f.i, f.literal(10));
+    final RexNode iNe30 = f.ne(f.i, f.literal(30));
+
+    f.checkImplies(iEq30, iEq30);
+    f.checkImplies(iNe10, iNe10);
+    f.checkImplies(iEq30, iNe10);
+    f.checkNotImplies(iNe10, iEq30);
+    f.checkNotImplies(iNe30, iEq30);
   }
 
   // Simple Tests for DataTypes
@@ -129,39 +129,42 @@ public class RexImplicationCheckerTest {
 
   @Test public void testSimpleBoolean() {
     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));
+    final RexNode bEqTrue = f.eq(f.bl, f.rexBuilder.makeLiteral(true));
+    final RexNode bEqFalse = f.eq(f.bl, f.rexBuilder.makeLiteral(false));
 
     //TODO: Need to support false => true
-    //f.checkImplies(node2, node1);
-    f.checkNotImplies(node1, node2);
+    //f.checkImplies(bEqFalse, bEqTrue);
+    f.checkNotImplies(bEqTrue, bEqFalse);
   }
 
   @Test public void testSimpleLong() {
     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);
+    final RexNode xGeBig = f.ge(f.lg, f.longLiteral(324324L));
+    final RexNode xGtBigger = f.gt(f.lg, f.longLiteral(324325L));
+    final RexNode xGeBigger = f.ge(f.lg, f.longLiteral(324325L));
+
+    f.checkImplies(xGtBigger, xGeBig);
+    f.checkImplies(xGtBigger, xGeBigger);
+    f.checkImplies(xGeBigger, xGeBig);
+    f.checkNotImplies(xGeBig, xGtBigger);
   }
 
   @Test public void testSimpleShort() {
     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));
+    final RexNode xGe10 = f.ge(f.sh, f.shortLiteral((short) 10));
+    final RexNode xGe11 = f.ge(f.sh, f.shortLiteral((short) 11));
 
-    f.checkImplies(node2, node1);
-    f.checkNotImplies(node1, node2);
+    f.checkImplies(xGe11, xGe10);
+    f.checkNotImplies(xGe10, xGe11);
   }
 
   @Test public void testSimpleChar() {
     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"));
+    final RexNode xGeB = f.ge(f.ch, f.charLiteral("b"));
+    final RexNode xGeA = f.ge(f.ch, f.charLiteral("a"));
 
-    f.checkImplies(node1, node2);
-    f.checkNotImplies(node2, node1);
+    f.checkImplies(xGeB, xGeA);
+    f.checkNotImplies(xGeA, xGeB);
   }
 
   @Test public void testSimpleString() {
@@ -206,21 +209,21 @@ public class RexImplicationCheckerTest {
 
   @Test public void testSimpleBetween() {
     final Fixture f = new Fixture();
-    final RexNode node1 = f.ge(f.i, f.literal(30));
-    final RexNode node2 = f.lt(f.i, f.literal(70));
-    final RexNode node3 = f.and(node1, node2);
-    final RexNode node4 = f.ge(f.i, f.literal(50));
-    final RexNode node5 = f.lt(f.i, f.literal(60));
-    final RexNode node6 = f.and(node4, node5);
-
-    f.checkNotImplies(node3, node4);
-    f.checkNotImplies(node3, node5);
-    f.checkNotImplies(node3, node6);
-    f.checkNotImplies(node1, node6);
-    f.checkNotImplies(node2, node6);
-    f.checkImplies(node6, node3);
-    f.checkImplies(node6, node2);
-    f.checkImplies(node6, node1);
+    final RexNode iGe30 = f.ge(f.i, f.literal(30));
+    final RexNode iLt70 = f.lt(f.i, f.literal(70));
+    final RexNode iGe30AndLt70 = f.and(iGe30, iLt70);
+    final RexNode iGe50 = f.ge(f.i, f.literal(50));
+    final RexNode iLt60 = f.lt(f.i, f.literal(60));
+    final RexNode iGe50AndLt60 = f.and(iGe50, iLt60);
+
+    f.checkNotImplies(iGe30AndLt70, iGe50);
+    f.checkNotImplies(iGe30AndLt70, iLt60);
+    f.checkNotImplies(iGe30AndLt70, iGe50AndLt60);
+    f.checkNotImplies(iGe30, iGe50AndLt60);
+    f.checkNotImplies(iLt70, iGe50AndLt60);
+    f.checkImplies(iGe50AndLt60, iGe30AndLt70);
+    f.checkImplies(iGe50AndLt60, iLt70);
+    f.checkImplies(iGe50AndLt60, iGe30);
   }
 
   @Test public void testSimpleBetweenCornerCases() {
@@ -239,6 +242,24 @@ public class RexImplicationCheckerTest {
     f.checkImplies(f.and(node3, node4), f.and(node5, node6));
   }
 
+  /** Similar to {@link MaterializationTest#testAlias()}:
+   * {@code x > 1 OR (y > 2 AND z > 4)}
+   * implies
+   * {@code (y > 3 AND z > 5)}. */
+  @Test public void testOr() {
+    final Fixture f = new Fixture();
+    final RexNode xGt1 = f.gt(f.i, f.literal(1));
+    final RexNode yGt2 = f.gt(f.dec, f.literal(2));
+    final RexNode yGt3 = f.gt(f.dec, f.literal(3));
+    final RexNode zGt4 = f.gt(f.lg, f.literal(4));
+    final RexNode zGt5 = f.gt(f.lg, f.literal(5));
+    final RexNode yGt2AndZGt4 = f.and(yGt2, zGt4);
+    final RexNode yGt3AndZGt5 = f.and(yGt3, zGt5);
+    final RexNode or = f.or(xGt1, yGt2AndZGt4);
+    //f.checkNotImplies(or, yGt3AndZGt5);
+    f.checkImplies(yGt3AndZGt5, or);
+  }
+
   @Test public void testNotNull() {
     final Fixture f = new Fixture();
     final RexNode node1 = f.eq(f.str, f.rexBuilder.makeLiteral("en"));
@@ -247,21 +268,41 @@ public class RexImplicationCheckerTest {
     f.checkImplies(node1, node2);
     f.checkNotImplies(node2, node1);
     f.checkImplies(node3, node2);
-    //TODO: Tough one
-    //f.checkImplies(node2, node2);
+    f.checkImplies(node2, node2);
   }
 
   @Test public void testIsNull() {
     final Fixture f = new Fixture();
-    final RexNode node1 = f.eq(f.str, f.rexBuilder.makeLiteral("en"));
-    final RexNode node2 = f.notNull(f.str);
-    final RexNode node3 = f.isNull(f.str);
-    f.checkNotImplies(node2, node3);
-    f.checkNotImplies(node3, node2);
-    f.checkNotImplies(node1, node3);
-    f.checkNotImplies(node3, node1);
-    //TODO:
-    //f.checkImplies(node3, node3);
+    final RexNode sEqEn = f.eq(f.str, f.charLiteral("en"));
+    final RexNode sIsNotNull = f.notNull(f.str);
+    final RexNode sIsNull = f.isNull(f.str);
+    final RexNode iEq5 = f.eq(f.i, f.literal(5));
+    final RexNode iIsNull = f.isNull(f.i);
+    f.checkNotImplies(sIsNotNull, sIsNull);
+    f.checkNotImplies(sIsNull, sIsNotNull);
+    f.checkNotImplies(sEqEn, sIsNull);
+    f.checkNotImplies(sIsNull, sEqEn);
+    f.checkImplies(sEqEn, sIsNotNull); // "s = literal" implies "s is not null"
+    f.checkImplies(sIsNotNull, sIsNotNull); // "s is not null" implies "s is not null"
+    f.checkImplies(sIsNull, sIsNull); // "s is null" implies "s is null"
+
+    // "s is not null and y = 5" implies "s is not null"
+    f.checkImplies(f.and(sIsNotNull, iEq5), sIsNotNull);
+
+    // "y = 5 and s is not null" implies "s is not null"
+    f.checkImplies(f.and(iEq5, sIsNotNull), sIsNotNull);
+
+    // "y is not null" does not imply "s is not null"
+    f.checkNotImplies(iIsNull, sIsNotNull);
+
+    // "s is not null or i = 5" does not imply "s is not null"
+    f.checkNotImplies(f.or(sIsNotNull, iEq5), sIsNotNull);
+
+    // "s is not null" implies "s is not null or i = 5"
+    f.checkImplies(sIsNotNull, f.or(sIsNotNull, iEq5));
+
+    // "s is not null" implies "i = 5 or s is not null"
+    f.checkImplies(sIsNotNull, f.or(iEq5, sIsNotNull));
   }
 
   /** Contains all the nourishment a test case could possibly need.
@@ -412,6 +453,10 @@ public class RexImplicationCheckerTest {
       return rexBuilder.makeCall(SqlStdOperatorTable.AND, nodes);
     }
 
+    public RexNode or(RexNode... nodes) {
+      return rexBuilder.makeCall(SqlStdOperatorTable.OR, nodes);
+    }
+
     public RexNode longLiteral(long value) {
       return rexBuilder.makeLiteral(value, longRelDataType, true);
     }