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

calcite git commit: [CALCITE-1526] Use Strong to infer whether a predicate's inputs may be null

Repository: calcite
Updated Branches:
  refs/heads/master e641211fd -> 584432cac


[CALCITE-1526] Use Strong to infer whether a predicate's inputs may be null

The goal is to combine into one place the logic for deducing whether
expressions are null.

Also, add SqlKind for COALESCE and NULLIF functions,
and baton down a flapping JDBC adapter test.


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

Branch: refs/heads/master
Commit: 584432cacdd3cbe87c974e3657b3c93d811b65df
Parents: e641211
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Dec 8 10:43:48 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Dec 9 15:19:27 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/plan/RelOptUtil.java     |   4 +-
 .../calcite/plan/RexImplicationChecker.java     |  25 +--
 .../java/org/apache/calcite/plan/Strong.java    | 184 ++++++++++++++++---
 .../calcite/rel/metadata/RelMdPredicates.java   |  37 ++--
 .../java/org/apache/calcite/rex/RexUtil.java    |  87 +++++++--
 .../java/org/apache/calcite/sql/SqlKind.java    |  71 +++----
 .../calcite/sql/fun/SqlCoalesceFunction.java    |   5 +-
 .../calcite/sql/fun/SqlNullifFunction.java      |   5 +-
 .../calcite/sql/parser/SqlParserTest.java       |  18 +-
 .../calcite/sql/test/SqlOperatorBaseTest.java   |  27 +--
 .../apache/calcite/test/JdbcAdapterTest.java    |   6 +-
 .../apache/calcite/test/RelMetadataTest.java    |  35 ++++
 .../calcite/test/RexImplicationCheckerTest.java |  13 ++
 .../org/apache/calcite/test/RexProgramTest.java |  76 ++++++--
 14 files changed, 426 insertions(+), 167 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/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 6ab8072..7d50d52 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -2233,11 +2233,11 @@ public abstract class RelOptUtil {
 
     for (RexNode filter : aboveFilters) {
       if (joinType.generatesNullsOnLeft()
-          && Strong.is(filter, leftBitmap)) {
+          && Strong.isNull(filter, leftBitmap)) {
         joinType = joinType.cancelNullsOnLeft();
       }
       if (joinType.generatesNullsOnRight()
-          && Strong.is(filter, rightBitmap)) {
+          && Strong.isNull(filter, rightBitmap)) {
         joinType = joinType.cancelNullsOnRight();
       }
       if (joinType == JoinRelType.INNER) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/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 056bffe..ff6c347 100644
--- a/core/src/main/java/org/apache/calcite/plan/RexImplicationChecker.java
+++ b/core/src/main/java/org/apache/calcite/plan/RexImplicationChecker.java
@@ -174,20 +174,21 @@ public class RexImplicationChecker {
     // Several things imply "IS NOT NULL"
     switch (second.getKind()) {
     case IS_NOT_NULL:
+      // Suppose we know that first is strong in second; that is,
+      // the if second is null, then first will be null.
+      // Then, first being not null implies that second is not null.
+      //
+      // For example, first is "x > y", second is "x".
+      // If we know that "x > y" is not null, we know that "x" 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 Strong strong = new Strong() {
+        @Override public boolean isNull(RexNode node) {
+          return RexUtil.eq(node, operand)
+              || super.isNull(node);
         }
+      };
+      if (strong.isNull(first)) {
+        return true;
       }
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/core/src/main/java/org/apache/calcite/plan/Strong.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/Strong.java b/core/src/main/java/org/apache/calcite/plan/Strong.java
index 5957bc7..b18e11c 100644
--- a/core/src/main/java/org/apache/calcite/plan/Strong.java
+++ b/core/src/main/java/org/apache/calcite/plan/Strong.java
@@ -20,9 +20,15 @@ import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.util.ImmutableBitSet;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import java.util.EnumMap;
 import java.util.List;
+import java.util.Map;
 
 /** Utilities for strong predicates.
  *
@@ -33,71 +39,205 @@ import java.util.List;
  *
  * <p>Examples:</p>
  * <ul>
- *   <li>{@code UNKNOWN} is strong
- *   <li>{@code c = 1} is strong
- *   <li>{@code c IS NULL} is not strong. (It always returns TRUE or FALSE.)
- *   <li>{@code p1 AND p2} is strong if p1 or p2 are strong
+ *   <li>{@code UNKNOWN} is strong in [] (definitely null)
+ *   <li>{@code c = 1} is strong in [c] (definitely null if and only if c is
+ *   null)
+ *   <li>{@code c IS NULL} is not strong (always returns TRUE or FALSE, never
+ *   null)
+ *   <li>{@code p1 AND p2} is strong in [p1, p2] (definitely null if either p1
+ *   is null or p2 is null)
  *   <li>{@code p1 OR p2} is strong if p1 and p2 are strong
- *   <li>{@code c1 = 1 OR c2 IS NULL} is strong on c1 but not c2
+ *   <li>{@code c1 = 1 OR c2 IS NULL} is strong in [c1] (definitely null if c1
+ *   is null)
  * </ul>
  */
 public class Strong {
-  private final ImmutableBitSet nullColumns;
+  private static final Map<SqlKind, Policy> MAP = createPolicyMap();
 
-  private Strong(ImmutableBitSet nullColumns) {
-    this.nullColumns = nullColumns;
+  /** Returns a checker that consults a bit set to find out whether particular
+   * inputs may be null. */
+  public static Strong of(final ImmutableBitSet nullColumns) {
+    return new Strong() {
+      @Override public boolean isNull(RexInputRef ref) {
+        return nullColumns.get(ref.getIndex());
+      }
+    };
   }
 
-  public static Strong of(ImmutableBitSet nullColumns) {
-    return new Strong(nullColumns);
+  /** Returns whether the analyzed expression will definitely return null if
+   * all of a given set of input columns are null. */
+  public static boolean isNull(RexNode node, ImmutableBitSet nullColumns) {
+    return of(nullColumns).isNull(node);
   }
 
-  /** Returns whether the analyzed expression will return null if a given set
-   * of input columns are null. */
-  public static boolean is(RexNode node, ImmutableBitSet nullColumns) {
-    return of(nullColumns).strong(node);
+  /** Returns how to deduce whether a particular kind of expression is null,
+   * given whether its arguments are null. */
+  public static Policy policy(SqlKind kind) {
+    return Preconditions.checkNotNull(MAP.get(kind), kind);
   }
 
-  private boolean strong(RexNode node) {
+  /** Returns whether an expression is definitely null.
+   *
+   * <p>The answer is based on calls to {@link #isNull} for its constituent
+   * expressions, and you may override methods to test hypotheses such as
+   * "if {@code x} is null, is {@code x + y} null? */
+  public boolean isNull(RexNode node) {
     switch (node.getKind()) {
     case LITERAL:
       return ((RexLiteral) node).getValue() == null;
     case IS_TRUE:
     case IS_NOT_NULL:
     case AND:
+    case NOT:
     case EQUALS:
     case NOT_EQUALS:
     case LESS_THAN:
     case LESS_THAN_OR_EQUAL:
     case GREATER_THAN:
     case GREATER_THAN_OR_EQUAL:
-      return anyStrong(((RexCall) node).getOperands());
+    case PLUS_PREFIX:
+    case MINUS_PREFIX:
+    case PLUS:
+    case TIMESTAMP_ADD:
+    case MINUS:
+    case TIMESTAMP_DIFF:
+    case TIMES:
+    case DIVIDE:
+    case CAST:
+    case REINTERPRET:
+    case TRIM:
+    case LTRIM:
+    case RTRIM:
+    case CEIL:
+    case FLOOR:
+    case EXTRACT:
+    case GREATEST:
+    case LEAST:
+      return anyNull(((RexCall) node).getOperands());
     case OR:
-      return allStrong(((RexCall) node).getOperands());
+      return allNull(((RexCall) node).getOperands());
     case INPUT_REF:
-      return nullColumns.get(((RexInputRef) node).getIndex());
+      return isNull((RexInputRef) node);
     default:
       return false;
     }
   }
 
-  private boolean allStrong(List<RexNode> operands) {
+  /** Returns whether a given input is definitely null. */
+  public boolean isNull(RexInputRef ref) {
+    return false;
+  }
+
+  /** Returns whether all expressions in a list are definitely null. */
+  private boolean allNull(List<RexNode> operands) {
     for (RexNode operand : operands) {
-      if (!strong(operand)) {
+      if (!isNull(operand)) {
         return false;
       }
     }
     return true;
   }
 
-  private boolean anyStrong(List<RexNode> operands) {
+  /** Returns whether any expressions in a list are definitely null. */
+  private boolean anyNull(List<RexNode> operands) {
     for (RexNode operand : operands) {
-      if (strong(operand)) {
+      if (isNull(operand)) {
         return true;
       }
     }
     return false;
   }
+
+  private static Map<SqlKind, Policy> createPolicyMap() {
+    EnumMap<SqlKind, Policy> map = new EnumMap<>(SqlKind.class);
+
+    map.put(SqlKind.INPUT_REF, Policy.AS_IS);
+    map.put(SqlKind.LOCAL_REF, Policy.AS_IS);
+    map.put(SqlKind.DYNAMIC_PARAM, Policy.AS_IS);
+    map.put(SqlKind.OTHER_FUNCTION, Policy.AS_IS);
+
+    // The following types of expressions could potentially be custom.
+    map.put(SqlKind.CASE, Policy.AS_IS);
+    map.put(SqlKind.DECODE, Policy.AS_IS);
+    // NULLIF(1, NULL) yields 1, but NULLIF(1, 1) yields NULL
+    map.put(SqlKind.NULLIF, Policy.AS_IS);
+    // COALESCE(NULL, 2) yields 2
+    map.put(SqlKind.COALESCE, Policy.AS_IS);
+    map.put(SqlKind.NVL, Policy.AS_IS);
+    // FALSE OR NULL yields FALSE
+    map.put(SqlKind.AND, Policy.AS_IS);
+    // TRUE OR NULL yields TRUE
+    map.put(SqlKind.OR, Policy.AS_IS);
+
+    // Expression types with custom handlers.
+    map.put(SqlKind.LITERAL, Policy.CUSTOM);
+
+    map.put(SqlKind.EXISTS, Policy.NOT_NULL);
+    map.put(SqlKind.IS_DISTINCT_FROM, Policy.NOT_NULL);
+    map.put(SqlKind.IS_NOT_DISTINCT_FROM, Policy.NOT_NULL);
+    map.put(SqlKind.IS_NULL, Policy.NOT_NULL);
+    map.put(SqlKind.IS_NOT_NULL, Policy.NOT_NULL);
+    map.put(SqlKind.IS_TRUE, Policy.NOT_NULL);
+    map.put(SqlKind.IS_NOT_TRUE, Policy.NOT_NULL);
+    map.put(SqlKind.IS_FALSE, Policy.NOT_NULL);
+    map.put(SqlKind.IS_NOT_FALSE, Policy.NOT_NULL);
+    map.put(SqlKind.IS_DISTINCT_FROM, Policy.NOT_NULL);
+    map.put(SqlKind.IS_NOT_DISTINCT_FROM, Policy.NOT_NULL);
+
+    map.put(SqlKind.NOT, Policy.ANY);
+    map.put(SqlKind.EQUALS, Policy.ANY);
+    map.put(SqlKind.NOT_EQUALS, Policy.ANY);
+    map.put(SqlKind.LESS_THAN, Policy.ANY);
+    map.put(SqlKind.LESS_THAN_OR_EQUAL, Policy.ANY);
+    map.put(SqlKind.GREATER_THAN, Policy.ANY);
+    map.put(SqlKind.GREATER_THAN_OR_EQUAL, Policy.ANY);
+    map.put(SqlKind.LIKE, Policy.ANY);
+    map.put(SqlKind.SIMILAR, Policy.ANY);
+    map.put(SqlKind.PLUS_PREFIX, Policy.ANY);
+    map.put(SqlKind.MINUS_PREFIX, Policy.ANY);
+    map.put(SqlKind.PLUS, Policy.ANY);
+    map.put(SqlKind.MINUS, Policy.ANY);
+    map.put(SqlKind.TIMES, Policy.ANY);
+    map.put(SqlKind.DIVIDE, Policy.ANY);
+    map.put(SqlKind.CAST, Policy.ANY);
+    map.put(SqlKind.REINTERPRET, Policy.ANY);
+    map.put(SqlKind.TRIM, Policy.ANY);
+    map.put(SqlKind.LTRIM, Policy.ANY);
+    map.put(SqlKind.RTRIM, Policy.ANY);
+    map.put(SqlKind.CEIL, Policy.ANY);
+    map.put(SqlKind.FLOOR, Policy.ANY);
+    map.put(SqlKind.EXTRACT, Policy.ANY);
+    map.put(SqlKind.GREATEST, Policy.ANY);
+    map.put(SqlKind.LEAST, Policy.ANY);
+
+    // Assume that any other expressions cannot be simplified.
+    for (SqlKind k
+        : Iterables.concat(SqlKind.EXPRESSION, SqlKind.AGGREGATE)) {
+      if (!map.containsKey(k)) {
+        map.put(k, Policy.AS_IS);
+      }
+    }
+    return map;
+  }
+
+  /** How whether an operator's operands are null affects whether a call to
+   * that operator evaluates to null. */
+  public enum Policy {
+    /** This kind of expression is never null. No need to look at its arguments,
+     * if it has any. */
+    NOT_NULL,
+
+    /** This kind of expression has its own particular rules about whether it
+     * is null. */
+    CUSTOM,
+
+    /** This kind of expression is null if and only if at least one of its
+     * arguments is null. */
+    ANY,
+
+    /** This kind of expression may be null. There is no way to rewrite. */
+    AS_IS,
+  }
 }
 
 // End Strong.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/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 df7e679..bfad3e7 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
@@ -21,6 +21,7 @@ import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.function.Predicate1;
 import org.apache.calcite.plan.RelOptPredicateList;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.Strong;
 import org.apache.calcite.plan.hep.HepRelVertex;
 import org.apache.calcite.plan.volcano.RelSubset;
 import org.apache.calcite.rel.RelNode;
@@ -194,7 +195,7 @@ public class RelMdPredicates
     // 'columnsMapped' construct a new predicate based on mapping.
     final ImmutableBitSet columnsMapped = columnsMappedBuilder.build();
     for (RexNode r : inputInfo.pulledUpPredicates) {
-      RexNode r2 = projectPredicate(rexBuilder, r, columnsMapped);
+      RexNode r2 = projectPredicate(rexBuilder, input, r, columnsMapped);
       if (!r2.isAlwaysTrue()) {
         r2 = r2.accept(new RexPermuteInputsShuttle(m, input));
         projectPullUpPredicates.add(r2);
@@ -229,6 +230,8 @@ public class RelMdPredicates
    * <p>Examples:<ol>
    * <li>The predicate {@code $7 = $9} on columns [7]
    *     becomes {@code $7 is not null}
+   * <li>The predicate {@code $7 = $9 + $11} on columns [7, 9]
+   *     becomes {@code $7 is not null or $9 is not null}
    * <li>The predicate {@code $7 = $9 and $9 = 5} on columns [7] becomes
    *   {@code $7 = 5}
    * <li>The predicate
@@ -236,36 +239,34 @@ public class RelMdPredicates
    *   on columns [7] becomes {@code $7 > 3}
    * </ol>
    *
-   * <p>We currently only handle example 1.
+   * <p>We currently only handle examples 1 and 2.
    *
+   * @param rexBuilder Rex builder
+   * @param input Input relational expression
    * @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) {
+  private RexNode projectPredicate(final RexBuilder rexBuilder, RelNode input,
+      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);
-          }
+      final List<RexNode> list = new ArrayList<>();
+      for (int c : columnsMapped.intersect(rCols)) {
+        if (input.getRowType().getFieldList().get(c).getType().isNullable()
+            && Strong.isNull(r, ImmutableBitSet.of(c))) {
+          list.add(
+              rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL,
+                  rexBuilder.makeInputRef(input, c)));
         }
       }
+      if (!list.isEmpty()) {
+        return RexUtil.composeDisjunction(rexBuilder, list, false);
+      }
     }
     // Cannot weaken to anything non-trivial
     return rexBuilder.makeLiteral(true);

http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/core/src/main/java/org/apache/calcite/rex/RexUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index 0301d7a..5fc194c 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -20,6 +20,7 @@ import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.function.Predicate1;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.Strong;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelFieldCollation;
@@ -1714,25 +1715,46 @@ public class RexUtil {
   private static RexNode simplifyIs(RexBuilder rexBuilder, RexCall call) {
     final SqlKind kind = call.getKind();
     final RexNode a = call.getOperands().get(0);
-    if (!a.getType().isNullable()) {
-      switch (kind) {
-      case IS_NULL:
-      case IS_NOT_NULL:
-        // x IS NULL ==> FALSE (if x is not nullable)
-        // x IS NOT NULL ==> TRUE (if x is not nullable)
-        return rexBuilder.makeLiteral(kind == SqlKind.IS_NOT_NULL);
-      case IS_TRUE:
-      case IS_NOT_FALSE:
-        // x IS TRUE ==> x (if x is not nullable)
-        // x IS NOT FALSE ==> x (if x is not nullable)
+    final RexNode simplified = simplifyIs2(rexBuilder, kind, a);
+    if (simplified != null) {
+      return simplified;
+    }
+    return call;
+  }
+
+  private static RexNode simplifyIs2(RexBuilder rexBuilder, SqlKind kind,
+      RexNode a) {
+    switch (kind) {
+    case IS_NULL:
+      // x IS NULL ==> FALSE (if x is not nullable)
+      if (!a.getType().isNullable()) {
+        return rexBuilder.makeLiteral(false);
+      }
+      break;
+    case IS_NOT_NULL:
+      // x IS NOT NULL ==> TRUE (if x is not nullable)
+      RexNode simplified = simplifyIsNotNull(rexBuilder, a);
+      if (simplified != null) {
+        return simplified;
+      }
+      break;
+    case IS_TRUE:
+    case IS_NOT_FALSE:
+      // x IS TRUE ==> x (if x is not nullable)
+      // x IS NOT FALSE ==> x (if x is not nullable)
+      if (!a.getType().isNullable()) {
         return simplify(rexBuilder, a);
-      case IS_FALSE:
-      case IS_NOT_TRUE:
-        // x IS NOT TRUE ==> NOT x (if x is not nullable)
-        // x IS FALSE ==> NOT x (if x is not nullable)
+      }
+      break;
+    case IS_FALSE:
+    case IS_NOT_TRUE:
+      // x IS NOT TRUE ==> NOT x (if x is not nullable)
+      // x IS FALSE ==> NOT x (if x is not nullable)
+      if (!a.getType().isNullable()) {
         return simplify(rexBuilder,
             rexBuilder.makeCall(SqlStdOperatorTable.NOT, a));
       }
+      break;
     }
     switch (a.getKind()) {
     case NOT:
@@ -1750,7 +1772,40 @@ public class RexUtil {
     if (a != a2) {
       return rexBuilder.makeCall(op(kind), ImmutableList.of(a2));
     }
-    return call;
+    return null; // cannot be simplified
+  }
+
+  private static RexNode simplifyIsNotNull(RexBuilder rexBuilder, RexNode a) {
+    if (!a.getType().isNullable()) {
+      return rexBuilder.makeLiteral(true);
+    }
+    switch (Strong.policy(a.getKind())) {
+    case ANY:
+      final List<RexNode> operands = new ArrayList<>();
+      for (RexNode operand : ((RexCall) a).getOperands()) {
+        final RexNode simplified = simplifyIsNotNull(rexBuilder, operand);
+        if (simplified == null) {
+          operands.add(
+              rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, operand));
+        } else if (simplified.isAlwaysFalse()) {
+          return rexBuilder.makeLiteral(false);
+        } else {
+          operands.add(simplified);
+        }
+      }
+      return composeConjunction(rexBuilder, operands, false);
+    case CUSTOM:
+      switch (a.getKind()) {
+      case LITERAL:
+        return rexBuilder.makeLiteral(((RexLiteral) a).getValue() != null);
+      default:
+        throw new AssertionError("every CUSTOM policy needs a handler, "
+            + a.getKind());
+      }
+    case AS_IS:
+    default:
+      return null;
+    }
   }
 
   private static SqlOperator op(SqlKind kind) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/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 cba968f..3300f00 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -853,41 +853,8 @@ public enum SqlKind {
       EnumSet.of(COUNT, SUM, SUM0, MIN, MAX, LEAD, LAG, FIRST_VALUE,
           LAST_VALUE, COVAR_POP, COVAR_SAMP, REGR_SXX, REGR_SYY,
           AVG, STDDEV_POP, STDDEV_SAMP, VAR_POP, VAR_SAMP, NTILE, COLLECT,
-          FUSION, SINGLE_VALUE);
-
-  /**
-   * Category consisting of all expression operators.
-   *
-   * <p>A node is an expression if it is NOT one of the following:
-   * {@link #AS},
-   * {@link #ARGUMENT_ASSIGNMENT},
-   * {@link #DEFAULT},
-   * {@link #DESCENDING},
-   * {@link #SELECT},
-   * {@link #JOIN},
-   * {@link #OTHER_FUNCTION},
-   * {@link #CAST},
-   * {@link #TRIM},
-   * {@link #LITERAL_CHAIN},
-   * {@link #JDBC_FN},
-   * {@link #PRECEDING},
-   * {@link #FOLLOWING},
-   * {@link #ORDER_BY},
-   * {@link #COLLECTION_TABLE},
-   * {@link #TABLESAMPLE},
-   * or an aggregate function.
-   */
-  public static final Set<SqlKind> EXPRESSION =
-      EnumSet.complementOf(
-          concat(
-              EnumSet.of(AS, ARGUMENT_ASSIGNMENT, DEFAULT,
-                  DESCENDING, CUBE, ROLLUP, GROUPING_SETS, EXTEND, LATERAL,
-                  SELECT, JOIN, OTHER_FUNCTION, CAST, TRIM, FLOOR, CEIL,
-                  TIMESTAMP_ADD, TIMESTAMP_DIFF, EXTRACT,
-                  LITERAL_CHAIN, JDBC_FN, PRECEDING, FOLLOWING, ORDER_BY,
-                  NULLS_FIRST, NULLS_LAST, COLLECTION_TABLE, TABLESAMPLE,
-                  VALUES, WITH, WITH_ITEM),
-              AGGREGATE));
+          FUSION, SINGLE_VALUE, ROW_NUMBER, RANK, PERCENT_RANK, DENSE_RANK,
+          CUME_DIST);
 
   /**
    * Category consisting of all DML operators.
@@ -936,6 +903,40 @@ public enum SqlKind {
           EXPLICIT_TABLE);
 
   /**
+   * Category consisting of all expression operators.
+   *
+   * <p>A node is an expression if it is NOT one of the following:
+   * {@link #AS},
+   * {@link #ARGUMENT_ASSIGNMENT},
+   * {@link #DEFAULT},
+   * {@link #DESCENDING},
+   * {@link #SELECT},
+   * {@link #JOIN},
+   * {@link #OTHER_FUNCTION},
+   * {@link #CAST},
+   * {@link #TRIM},
+   * {@link #LITERAL_CHAIN},
+   * {@link #JDBC_FN},
+   * {@link #PRECEDING},
+   * {@link #FOLLOWING},
+   * {@link #ORDER_BY},
+   * {@link #COLLECTION_TABLE},
+   * {@link #TABLESAMPLE},
+   * or an aggregate function, DML or DDL.
+   */
+  public static final Set<SqlKind> EXPRESSION =
+      EnumSet.complementOf(
+          concat(
+              EnumSet.of(AS, ARGUMENT_ASSIGNMENT, DEFAULT,
+                  DESCENDING, CUBE, ROLLUP, GROUPING_SETS, EXTEND, LATERAL,
+                  SELECT, JOIN, OTHER_FUNCTION, CAST, TRIM, FLOOR, CEIL,
+                  TIMESTAMP_ADD, TIMESTAMP_DIFF, EXTRACT,
+                  LITERAL_CHAIN, JDBC_FN, PRECEDING, FOLLOWING, ORDER_BY,
+                  NULLS_FIRST, NULLS_LAST, COLLECTION_TABLE, TABLESAMPLE,
+                  VALUES, WITH, WITH_ITEM),
+              AGGREGATE, DML, DDL));
+
+  /**
    * Category of all SQL statement types.
    *
    * <p>Consists of all types in {@link #QUERY}, {@link #DML} and {@link #DDL}.

http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
index 89f33af..fc8d3c8 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
@@ -42,9 +42,8 @@ public class SqlCoalesceFunction extends SqlFunction {
     // rewriteCall to convert COALESCE into CASE early.  However,
     // validator rewrite can optionally be disabled, in which case these
     // strategies are used.
-    super(
-        "COALESCE",
-        SqlKind.OTHER_FUNCTION,
+    super("COALESCE",
+        SqlKind.COALESCE,
         ReturnTypes.LEAST_RESTRICTIVE,
         null,
         OperandTypes.SAME_VARIADIC,

http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java
index c5529f0..0aa310e 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java
@@ -42,9 +42,8 @@ public class SqlNullifFunction extends SqlFunction {
     // rewriteCall to convert NULLIF into CASE early.  However,
     // validator rewrite can optionally be disabled, in which case these
     // strategies are used.
-    super(
-        "NULLIF",
-        SqlKind.OTHER_FUNCTION,
+    super("NULLIF",
+        SqlKind.NULLIF,
         ReturnTypes.ARG0_FORCE_NULLABLE,
         null,
         OperandTypes.COMPARABLE_UNORDERED_COMPARABLE_UNORDERED,

http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index fa12def..e7a90a1 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -3468,7 +3468,7 @@ public class SqlParserTest {
   @Test public void testNullIf() {
     checkExp(
         "nullif(v1,v2)",
-        "NULLIF(`V1`, `V2`)");
+        "(NULLIF(`V1`, `V2`))");
     checkExpFails(
         "1 ^+^ nullif + 3",
         "(?s)Encountered \"\\+ nullif \\+\" at line 1, column 3.*");
@@ -3477,13 +3477,13 @@ public class SqlParserTest {
   @Test public void testCoalesce() {
     checkExp(
         "coalesce(v1)",
-        "COALESCE(`V1`)");
+        "(COALESCE(`V1`))");
     checkExp(
         "coalesce(v1,v2)",
-        "COALESCE(`V1`, `V2`)");
+        "(COALESCE(`V1`, `V2`))");
     checkExp(
         "coalesce(v1,v2,v3)",
-        "COALESCE(`V1`, `V2`, `V3`)");
+        "(COALESCE(`V1`, `V2`, `V3`))");
   }
 
   @Test public void testLiteralCollate() {
@@ -3638,8 +3638,8 @@ public class SqlParserTest {
     checkExp(
         "trim (coalesce(cast(null as varchar(2)))||"
             + "' '||coalesce('junk ',''))",
-        "TRIM(BOTH ' ' FROM ((COALESCE(CAST(NULL AS VARCHAR(2))) || "
-            + "' ') || COALESCE('junk ', '')))");
+        "TRIM(BOTH ' ' FROM (((COALESCE(CAST(NULL AS VARCHAR(2)))) || "
+            + "' ') || (COALESCE('junk ', ''))))");
 
     checkFails(
         "trim(^from^ 'beard')",
@@ -6700,9 +6700,9 @@ public class SqlParserTest {
   }
 
   @Test public void testProcedureCall() {
-    check("call blubber(5)", "(CALL `BLUBBER`(5))");
-    check("call \"blubber\"(5)", "(CALL `blubber`(5))");
-    check("call whale.blubber(5)", "(CALL `WHALE`.`BLUBBER`(5))");
+    check("call blubber(5)", "CALL `BLUBBER`(5)");
+    check("call \"blubber\"(5)", "CALL `blubber`(5)");
+    check("call whale.blubber(5)", "CALL `WHALE`.`BLUBBER`(5)");
   }
 
   @Test public void testNewSpecification() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
index d2bd146..ef60d10 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
@@ -18,6 +18,7 @@ package org.apache.calcite.sql.test;
 
 import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.plan.Strong;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.runtime.Hook;
@@ -5914,9 +5915,9 @@ public abstract class SqlOperatorBaseTest {
               || s.matches("MOD\\(.*, 0\\)")) {
             continue;
           }
-          boolean strict = isStrict(op);
+          final Strong.Policy policy = Strong.policy(op.kind);
           try {
-            if (nullCount > 0 && strict) {
+            if (nullCount > 0 && policy == Strong.Policy.ANY) {
               tester.checkNull(s);
             } else {
               final String query;
@@ -5943,28 +5944,6 @@ public abstract class SqlOperatorBaseTest {
     }
   }
 
-  /** Returns whether an operator always returns null if any of its arguments is
-   * null. */
-  private static boolean isStrict(SqlOperator op) {
-    if (op == SqlStdOperatorTable.NULLIF) {
-      return false;
-    }
-    switch (op.kind) {
-    case IS_DISTINCT_FROM:
-    case IS_NOT_DISTINCT_FROM:
-    case IS_NULL:
-    case IS_NOT_NULL:
-    case IS_TRUE:
-    case IS_NOT_TRUE:
-    case IS_FALSE:
-    case IS_NOT_FALSE:
-    case AND: // not strict: FALSE OR NULL yields FALSE
-    case OR: // not strict: TRUE OR NULL yields TRUE
-      return false;
-    }
-    return true;
-  }
-
   private List<Object> getValues(BasicSqlType type, boolean inBound) {
     List<Object> values = new ArrayList<Object>();
     for (boolean sign : FALSE_TRUE) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
index 2efea7b..ccff1c8 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
@@ -662,17 +662,17 @@ public class JdbcAdapterTest {
         + " \"category_id\", \"currency_id\", \"amount\")\n"
         + "VALUES (666, 666, TIMESTAMP '1997-01-01 00:00:00',"
         + "   666, '666', 666, 666),\n"
-        + " (777, 666, TIMESTAMP '1997-01-01 00:00:00',"
+        + " (666, 777, TIMESTAMP '1997-01-01 00:00:00',"
         + "   666, '666', 666, 666)";
     final String explain = "PLAN=JdbcToEnumerableConverter\n"
         + "  JdbcTableModify(table=[[foodmart, expense_fact]], operation=[INSERT], flattened=[false])\n"
         + "    JdbcValues(tuples=[[{ 666, 666, 1997-01-01 00:00:00, 666, '666', 666, 666.0000 },"
-        + " { 777, 666, 1997-01-01 00:00:00, 666, '666', 666, 666.0000 }]])\n";
+        + " { 666, 777, 1997-01-01 00:00:00, 666, '666', 666, 666.0000 }]])\n";
     final String jdbcSql = "INSERT INTO \"foodmart\".\"expense_fact\""
         + " (\"store_id\", \"account_id\", \"exp_date\", \"time_id\","
         + " \"category_id\", \"currency_id\", \"amount\")\n"
         + "VALUES  (666, 666, TIMESTAMP '1997-01-01 00:00:00', 666, '666', 666, 666.0000),\n"
-        + " (777, 666, TIMESTAMP '1997-01-01 00:00:00', 666, '666', 666, 666.0000)";
+        + " (666, 777, TIMESTAMP '1997-01-01 00:00:00', 666, '666', 666, 666.0000)";
     CalciteAssert
         .model(JdbcTest.FOODMART_MODEL)
         .query(sql)

http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/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 df2bd9c..f11b26a 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -1372,6 +1372,41 @@ public class RelMetadataTest extends SqlToRelTestBase {
     assertThat(predicates.pulledUpPredicates, sortsAs("[IS NOT NULL($0)]"));
     assertThat(predicates.leftInferredPredicates.isEmpty(), is(true));
     assertThat(predicates.rightInferredPredicates.isEmpty(), is(true));
+
+    // Create another similar Join. From the join condition
+    //   e.MGR - e.EMPNO = d.DEPTNO + e.MGR_COMM
+    // we can deduce the projected predicate
+    //   MGR IS NOT NULL OR MGR_COMM IS NOT NULL
+    //
+    // EMPNO is omitted because it is NOT NULL.
+    // MGR_COMM is a made-up nullable field.
+    relBuilder.push(filter);
+    relBuilder.project(
+        Iterables.concat(relBuilder.fields(),
+            ImmutableList.of(
+                relBuilder.alias(
+                    relBuilder.call(SqlStdOperatorTable.PLUS,
+                        relBuilder.field("MGR"),
+                        relBuilder.field("COMM")),
+                    "MGR_COMM"))));
+    relBuilder.push(deptScan);
+    relBuilder.join(JoinRelType.INNER,
+        relBuilder.equals(
+            relBuilder.call(SqlStdOperatorTable.MINUS,
+                relBuilder.field(2, 0, "MGR"),
+                relBuilder.field(2, 0, "EMPNO")),
+            relBuilder.call(SqlStdOperatorTable.PLUS,
+                relBuilder.field(2, 1, "DEPTNO"),
+                relBuilder.field(2, 0, "MGR_COMM"))));
+
+    relBuilder.project(relBuilder.field("MGR"), relBuilder.field("NAME"),
+        relBuilder.field("MGR_COMM"), relBuilder.field("COMM"));
+    final RelNode project3 = relBuilder.peek();
+    predicates = mq.getPulledUpPredicates(project3);
+    assertThat(predicates.pulledUpPredicates,
+        sortsAs("[OR(IS NOT NULL($0), IS NOT NULL($2))]"));
+    assertThat(predicates.leftInferredPredicates.isEmpty(), is(true));
+    assertThat(predicates.rightInferredPredicates.isEmpty(), is(true));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/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 4464019..69cccd0 100644
--- a/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
@@ -278,6 +278,7 @@ public class RexImplicationCheckerTest {
     final RexNode sIsNull = f.isNull(f.str);
     final RexNode iEq5 = f.eq(f.i, f.literal(5));
     final RexNode iIsNull = f.isNull(f.i);
+    final RexNode iIsNotNull = f.notNull(f.i);
     f.checkNotImplies(sIsNotNull, sIsNull);
     f.checkNotImplies(sIsNull, sIsNotNull);
     f.checkNotImplies(sEqEn, sIsNull);
@@ -303,6 +304,18 @@ public class RexImplicationCheckerTest {
 
     // "s is not null" implies "i = 5 or s is not null"
     f.checkImplies(sIsNotNull, f.or(iEq5, sIsNotNull));
+
+    // "i > 10" implies "x is not null"
+    f.checkImplies(f.gt(f.i, f.literal(10)), iIsNotNull);
+
+    // "-20 > i" implies "x is not null"
+    f.checkImplies(f.gt(f.literal(-20), f.i), iIsNotNull);
+
+    // "s is null and -20 > i" implies "x is not null"
+    f.checkImplies(f.and(sIsNull, f.gt(f.literal(-20), f.i)), iIsNotNull);
+
+    // "i > 10" does not imply "x is null"
+    f.checkNotImplies(f.gt(f.i, f.literal(10)), iIsNull);
   }
 
   /** Contains all the nourishment a test case could possibly need.

http://git-wip-us.apache.org/repos/asf/calcite/blob/584432ca/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
index 641346d..f72ab55 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -154,6 +154,14 @@ public class RexProgramTest {
     return n;
   }
 
+  private RexNode isNull(RexNode node) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, node);
+  }
+
+  private RexNode isNotNull(RexNode node) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, node);
+  }
+
   private RexNode not(RexNode node) {
     return rexBuilder.makeCall(SqlStdOperatorTable.NOT, node);
   }
@@ -481,10 +489,6 @@ public class RexProgramTest {
     return builder;
   }
 
-  static boolean strongIf(RexNode e, ImmutableBitSet b) {
-    return Strong.is(e, b);
-  }
-
   /** Unit test for {@link org.apache.calcite.plan.Strong}. */
   @Test public void testStrong() {
     final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
@@ -496,29 +500,42 @@ public class RexProgramTest {
     final ImmutableBitSet c13 = ImmutableBitSet.of(1, 3);
 
     // input ref
-    final RexInputRef aRef = rexBuilder.makeInputRef(intType, 0);
+    final RexInputRef i0 = rexBuilder.makeInputRef(intType, 0);
+    final RexInputRef i1 = rexBuilder.makeInputRef(intType, 1);
 
-    assertThat(strongIf(aRef, c0), is(true));
-    assertThat(strongIf(aRef, c1), is(false));
-    assertThat(strongIf(aRef, c01), is(true));
-    assertThat(strongIf(aRef, c13), is(false));
+    assertThat(Strong.isNull(i0, c0), is(true));
+    assertThat(Strong.isNull(i0, c1), is(false));
+    assertThat(Strong.isNull(i0, c01), is(true));
+    assertThat(Strong.isNull(i0, c13), is(false));
 
     // literals are strong iff they are always null
-    assertThat(strongIf(trueLiteral, c), is(false));
-    assertThat(strongIf(trueLiteral, c13), is(false));
-    assertThat(strongIf(falseLiteral, c13), is(false));
-    assertThat(strongIf(nullLiteral, c), is(true));
-    assertThat(strongIf(nullLiteral, c13), is(true));
-    assertThat(strongIf(unknownLiteral, c13), is(true));
+    assertThat(Strong.isNull(trueLiteral, c), is(false));
+    assertThat(Strong.isNull(trueLiteral, c13), is(false));
+    assertThat(Strong.isNull(falseLiteral, c13), is(false));
+    assertThat(Strong.isNull(nullLiteral, c), is(true));
+    assertThat(Strong.isNull(nullLiteral, c13), is(true));
+    assertThat(Strong.isNull(unknownLiteral, c13), is(true));
 
     // AND is strong if one of its arguments is strong
     final RexNode andUnknownTrue = and(unknownLiteral, trueLiteral);
     final RexNode andTrueUnknown = and(trueLiteral, unknownLiteral);
     final RexNode andFalseTrue = and(falseLiteral, trueLiteral);
 
-    assertThat(strongIf(andUnknownTrue, c), is(true));
-    assertThat(strongIf(andTrueUnknown, c), is(true));
-    assertThat(strongIf(andFalseTrue, c), is(false));
+    assertThat(Strong.isNull(andUnknownTrue, c), is(true));
+    assertThat(Strong.isNull(andTrueUnknown, c), is(true));
+    assertThat(Strong.isNull(andFalseTrue, c), is(false));
+
+    // If i0 is null, "i0 and i1 is null" is null
+    assertThat(Strong.isNull(and(i0, isNull(i1)), c0), is(true));
+    // If i1 is null, "i0 and i1 is null" is not necessarily null
+    assertThat(Strong.isNull(and(i0, isNull(i1)), c1), is(false));
+    // If i0 and i1 are both null, "i0 and i1 is null" is null
+    assertThat(Strong.isNull(and(i0, isNull(i1)), c01), is(true));
+    // If i0 and i1 are both null, "i0 or i1" is null
+    assertThat(Strong.isNull(or(i0, i1), c01), is(true));
+    // If i0 is null, "i0 or i1" is not necessarily null
+    assertThat(Strong.isNull(or(i0, i1), c0), is(false));
+    assertThat(Strong.isNull(or(i0, i1), c1), is(false));
   }
 
   /** Unit test for {@link org.apache.calcite.rex.RexUtil#toCnf}. */
@@ -1081,6 +1098,27 @@ public class RexProgramTest {
     assertThat(result.getType().getSqlTypeName(), is(SqlTypeName.BOOLEAN));
   }
 
+  @Test public void testSimplifyIsNotNull() {
+    RelDataType intType =
+        typeFactory.createTypeWithNullability(
+            typeFactory.createSqlType(SqlTypeName.INTEGER), false);
+    RelDataType intNullableType =
+        typeFactory.createTypeWithNullability(
+            typeFactory.createSqlType(SqlTypeName.INTEGER), true);
+    final RexInputRef i0 = rexBuilder.makeInputRef(intNullableType, 0);
+    final RexInputRef i1 = rexBuilder.makeInputRef(intNullableType, 1);
+    final RexInputRef i2 = rexBuilder.makeInputRef(intType, 2);
+    final RexInputRef i3 = rexBuilder.makeInputRef(intType, 3);
+    final RexLiteral one = rexBuilder.makeExactLiteral(BigDecimal.ONE);
+    final RexNode null_ = rexBuilder.makeNullLiteral(SqlTypeName.INTEGER);
+    checkSimplify(isNotNull(lt(i0, i1)),
+        "AND(IS NOT NULL($0), IS NOT NULL($1))");
+    checkSimplify(isNotNull(lt(i0, i2)), "IS NOT NULL($0)");
+    checkSimplify(isNotNull(lt(i2, i3)), "true");
+    checkSimplify(isNotNull(lt(i0, one)), "IS NOT NULL($0)");
+    checkSimplify(isNotNull(lt(i0, null_)), "false");
+  }
+
   @Test public void testSimplifyCastLiteral() {
     final List<RexLiteral> literals = new ArrayList<>();
     literals.add(
@@ -1234,8 +1272,6 @@ public class RexProgramTest {
             ImmutableList.of(eq(aRef, literal1),
                 eq(aRef, literal2)));
     assertThat(getString(map3), is("{1=?0.a, 2=?0.a}"));
-
-
   }
 
   /** Converts a map to a string, sorting on the string representation of its