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/06/18 19:39:14 UTC

incubator-calcite git commit: [CALCITE-727] Constant folding involving CASE and NULL

Repository: incubator-calcite
Updated Branches:
  refs/heads/master 31cab4da1 -> 21cf1259b


[CALCITE-727] Constant folding involving CASE and NULL

Move a few functions into RexUtil, and make the simplify and andNot methods smarter.


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

Branch: refs/heads/master
Commit: 21cf1259b8c02b63dbfebb8cf23e65f73bca567c
Parents: 31cab4d
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jun 16 16:34:08 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Jun 18 01:32:13 2015 -0700

----------------------------------------------------------------------
 .../calcite/plan/SubstitutionVisitor.java       | 178 ++---------
 .../rel/rules/ReduceExpressionsRule.java        |  60 +++-
 .../java/org/apache/calcite/rex/RexCall.java    |   4 +
 .../apache/calcite/rex/RexProgramBuilder.java   |  26 +-
 .../java/org/apache/calcite/rex/RexUtil.java    | 300 ++++++++++++++++++-
 .../calcite/test/MaterializationTest.java       |   7 +-
 .../apache/calcite/test/RelOptRulesTest.java    |  65 ++++
 .../org/apache/calcite/test/RexProgramTest.java |  85 ++++++
 .../org/apache/calcite/test/RelOptRulesTest.xml | 101 ++++++-
 9 files changed, 627 insertions(+), 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/21cf1259/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 db69196..7322d4b 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -47,7 +47,6 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlAggFunction;
-import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.Bug;
@@ -79,13 +78,16 @@ import java.util.AbstractList;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import static org.apache.calcite.rex.RexUtil.andNot;
+import static org.apache.calcite.rex.RexUtil.removeAll;
+import static org.apache.calcite.rex.RexUtil.simplify;
+
 /**
  * Substitutes part of a tree of relational expressions with another tree.
  *
@@ -152,7 +154,7 @@ public class SubstitutionVisitor {
           AggregateOnProjectToAggregateUnifyRule.INSTANCE);
 
   private static final Map<Pair<Class, Class>, List<UnifyRule>> RULE_MAP =
-      new HashMap<Pair<Class, Class>, List<UnifyRule>>();
+      new HashMap<>();
 
   private final RelOptCluster cluster;
   private final Holder query;
@@ -168,8 +170,7 @@ public class SubstitutionVisitor {
    */
   final List<MutableRel> queryLeaves;
 
-  final Map<MutableRel, MutableRel> replacementMap =
-      new HashMap<MutableRel, MutableRel>();
+  final Map<MutableRel, MutableRel> replacementMap = new HashMap<>();
 
   final Multimap<MutableRel, MutableRel> equivalents =
       LinkedHashMultimap.create();
@@ -184,7 +185,7 @@ public class SubstitutionVisitor {
     this.query = Holder.of(toMutable(query_));
     this.target = toMutable(target_);
     final Set<MutableRel> parents = Sets.newIdentityHashSet();
-    final List<MutableRel> allNodes = new ArrayList<MutableRel>();
+    final List<MutableRel> allNodes = new ArrayList<>();
     final MutableRelVisitor visitor =
         new MutableRelVisitor() {
           public void visit(MutableRel node) {
@@ -332,45 +333,7 @@ public class SubstitutionVisitor {
       }
     }
     return RexUtil.composeConjunction(rexBuilder,
-        Lists.transform(targets, not(rexBuilder)), false);
-  }
-
-  /** Returns a function that applies NOT to its argument. */
-  public static Function<RexNode, RexNode> not(final RexBuilder rexBuilder) {
-    return new Function<RexNode, RexNode>() {
-      public RexNode apply(RexNode input) {
-        return input.isAlwaysTrue()
-            ? rexBuilder.makeLiteral(false)
-            : input.isAlwaysFalse()
-            ? rexBuilder.makeLiteral(true)
-            : input.getKind() == SqlKind.NOT
-            ? ((RexCall) input).operands.get(0)
-            : rexBuilder.makeCall(SqlStdOperatorTable.NOT, input);
-      }
-    };
-  }
-
-  /** Removes all expressions from a list that are equivalent to a given
-   * expression. Returns whether any were removed. */
-  private static boolean removeAll(List<RexNode> targets, RexNode e) {
-    int count = 0;
-    Iterator<RexNode> iterator = targets.iterator();
-    while (iterator.hasNext()) {
-      RexNode next = iterator.next();
-      if (equivalent(next, e)) {
-        ++count;
-        iterator.remove();
-      }
-    }
-    return count > 0;
-  }
-
-  /** Returns whether two expressions are equivalent. */
-  private static boolean equivalent(RexNode e1, RexNode e2) {
-    // TODO: make broader;
-    // 1. 'x = y' should be equivalent to 'y = x'.
-    // 2. 'c2 and c1' should be equivalent to 'c1 and c2'.
-    return e1 == e2 || e1.toString().equals(e2.toString());
+        Lists.transform(targets, RexUtil.notFn(rexBuilder)), false);
   }
 
   /**
@@ -385,8 +348,8 @@ public class SubstitutionVisitor {
     //  e: x = 1 AND y = 2 AND z = 3 AND NOT (x = 1 AND y = 2)
     //  disjunctions: {x = 1, y = 2, z = 3}
     //  notDisjunctions: {x = 1 AND y = 2}
-    final List<RexNode> disjunctions = new ArrayList<RexNode>();
-    final List<RexNode> notDisjunctions = new ArrayList<RexNode>();
+    final List<RexNode> disjunctions = new ArrayList<>();
+    final List<RexNode> notDisjunctions = new ArrayList<>();
     RelOptUtil.decomposeConjunction(e, disjunctions, notDisjunctions);
 
     // If there is a single FALSE or NOT TRUE, the whole expression is
@@ -424,78 +387,6 @@ public class SubstitutionVisitor {
     return true;
   }
 
-  /**
-   * Simplifies a boolean expression.
-   *
-   * <p>In particular:</p>
-   * <ul>
-   * <li>{@code simplify(x = 1 AND y = 2 AND NOT x = 1)}
-   * returns {@code y = 2}</li>
-   * <li>{@code simplify(x = 1 AND FALSE)}
-   * returns {@code FALSE}</li>
-   * </ul>
-   */
-  public static RexNode simplify(RexBuilder rexBuilder, RexNode e) {
-    final List<RexNode> disjunctions = RelOptUtil.conjunctions(e);
-    final List<RexNode> notDisjunctions = new ArrayList<RexNode>();
-    for (int i = 0; i < disjunctions.size(); i++) {
-      final RexNode disjunction = disjunctions.get(i);
-      final SqlKind kind = disjunction.getKind();
-      switch (kind) {
-      case NOT:
-        notDisjunctions.add(
-            ((RexCall) disjunction).getOperands().get(0));
-        disjunctions.remove(i);
-        --i;
-        break;
-      case LITERAL:
-        if (!RexLiteral.booleanValue(disjunction)) {
-          return disjunction; // false
-        } else {
-          disjunctions.remove(i);
-          --i;
-        }
-      }
-    }
-    if (disjunctions.isEmpty() && notDisjunctions.isEmpty()) {
-      return rexBuilder.makeLiteral(true);
-    }
-    // If one of the not-disjunctions is a disjunction that is wholly
-    // contained in the disjunctions list, the expression is not
-    // satisfiable.
-    //
-    // Example #1. x AND y AND z AND NOT (x AND y)  - not satisfiable
-    // Example #2. x AND y AND NOT (x AND y)        - not satisfiable
-    // Example #3. x AND y AND NOT (x AND y AND z)  - may be satisfiable
-    for (RexNode notDisjunction : notDisjunctions) {
-      final List<RexNode> disjunctions2 =
-          RelOptUtil.conjunctions(notDisjunction);
-      if (disjunctions.containsAll(disjunctions2)) {
-        return rexBuilder.makeLiteral(false);
-      }
-    }
-    // Add the NOT disjunctions back in.
-    for (RexNode notDisjunction : notDisjunctions) {
-      disjunctions.add(
-          rexBuilder.makeCall(
-              SqlStdOperatorTable.NOT,
-              notDisjunction));
-    }
-    return RexUtil.composeConjunction(rexBuilder, disjunctions, false);
-  }
-
-  /**
-   * Creates the expression {@code e1 AND NOT e2}.
-   */
-  static RexNode andNot(RexBuilder rexBuilder, RexNode e1, RexNode e2) {
-    return rexBuilder.makeCall(
-        SqlStdOperatorTable.AND,
-        e1,
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.NOT,
-            e2));
-  }
-
   public RelNode go0(RelNode replacement_) {
     assert false; // not called
     MutableRel replacement = toMutable(replacement_);
@@ -1011,7 +902,7 @@ public class SubstitutionVisitor {
             + "input: " + input + "\n"
             + "project: " + shuttle + "\n");
       }
-      final List<RexNode> exprList = new ArrayList<RexNode>();
+      final List<RexNode> exprList = new ArrayList<>();
       final RexBuilder rexBuilder = input.cluster.getRexBuilder();
       final List<RexNode> projects = Pair.left(namedProjects);
       for (RexNode expr : projects) {
@@ -1035,7 +926,7 @@ public class SubstitutionVisitor {
             + "input: " + input + "\n"
             + "project: " + project + "\n");
       }
-      final List<RexNode> exprList = new ArrayList<RexNode>();
+      final List<RexNode> exprList = new ArrayList<>();
       final RexBuilder rexBuilder = model.cluster.getRexBuilder();
       for (RelDataTypeField field : model.getRowType().getFieldList()) {
         exprList.add(rexBuilder.makeZeroLiteral(field.getType()));
@@ -1298,7 +1189,7 @@ public class SubstitutionVisitor {
   /** Builds a shuttle that stores a list of expressions, and can map incoming
    * expressions to references to them. */
   private static RexShuttle getRexShuttle(MutableProject target) {
-    final Map<String, Integer> map = new HashMap<String, Integer>();
+    final Map<String, Integer> map = new HashMap<>();
     for (RexNode e : target.getProjects()) {
       map.put(e.toString(), map.size());
     }
@@ -1879,35 +1770,17 @@ public class SubstitutionVisitor {
      */
     protected JoinRelType joinType;
 
-    //~ Constructors -----------------------------------------------------------
-
-    /**
-     * Creates a Join.
-     *
-     * @param cluster          Cluster
-     * @param traits           Traits
-     * @param left             Left input
-     * @param right            Right input
-     * @param condition        Join condition
-     * @param joinType         Join type
-     * @param variablesStopped Set of names of variables which are set by the
-     *                         LHS and used by the RHS and are not available to
-     *                         nodes above this LogicalJoin in the tree
-     */
     private MutableJoin(
-        RelOptCluster cluster,
         RelDataType rowType,
         MutableRel left,
         MutableRel right,
         RexNode condition,
         JoinRelType joinType,
         Set<String> variablesStopped) {
-      super(MutableRelType.JOIN, cluster, rowType, left, right);
-      this.condition = condition;
+      super(MutableRelType.JOIN, left.cluster, rowType, left, right);
+      this.condition = Preconditions.checkNotNull(condition);
       this.variablesStopped = ImmutableSet.copyOf(variablesStopped);
-      assert joinType != null;
-      assert condition != null;
-      this.joinType = joinType;
+      this.joinType = Preconditions.checkNotNull(joinType);
     }
 
     public RexNode getCondition() {
@@ -1922,19 +1795,21 @@ public class SubstitutionVisitor {
       return variablesStopped;
     }
 
-    static MutableJoin of(RelOptCluster cluster, MutableRel left, MutableRel right,
-        RexNode condition, JoinRelType joinType, Set<String> variablesStopped) {
+    static MutableJoin of(RelOptCluster cluster, MutableRel left,
+        MutableRel right, RexNode condition, JoinRelType joinType,
+        Set<String> variablesStopped) {
       List<RelDataTypeField> fieldList = Collections.emptyList();
       RelDataType rowType =
           Join.deriveJoinRowType(left.getRowType(), right.getRowType(),
               joinType, cluster.getTypeFactory(), null, fieldList);
-      return new MutableJoin(cluster, rowType, left, right,
-          condition, joinType, variablesStopped);
+      return new MutableJoin(rowType, left, right, condition, joinType,
+          variablesStopped);
     }
 
     @Override public StringBuilder digest(StringBuilder buf) {
-      return buf.append("Join(left: ").append(left).append(", right:")
-          .append(right).append(")");
+      return buf.append("Join(left: ").append(left)
+          .append(", right:").append(right)
+          .append(")");
     }
   }
 
@@ -1963,7 +1838,7 @@ public class SubstitutionVisitor {
     }
 
     private static List<MutableRel> descendants(MutableRel query) {
-      final List<MutableRel> list = new ArrayList<MutableRel>();
+      final List<MutableRel> list = new ArrayList<>();
       descendantsRecurse(list, query);
       return list;
     }
@@ -2231,8 +2106,7 @@ public class SubstitutionVisitor {
       final LogicalFilter filter = call.rel(0);
       final LogicalProject project = call.rel(1);
 
-      final List<RexNode> newProjects =
-          new ArrayList<RexNode>(project.getProjects());
+      final List<RexNode> newProjects = new ArrayList<>(project.getProjects());
       newProjects.add(filter.getCondition());
 
       final RelOptCluster cluster = filter.getCluster();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/21cf1259/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
index 8bcf4b2..5ab9192 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
@@ -52,6 +52,7 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlRowOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Stacks;
 import org.apache.calcite.util.Util;
@@ -60,6 +61,7 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -332,6 +334,17 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
       RelOptPredicateList predicates) {
     RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
 
+    // Replace predicates on CASE to CASE on predicates.
+    for (int i = 0; i < expList.size(); i++) {
+      RexNode exp = expList.get(i);
+      if (exp instanceof RexCall) {
+        RexNode exp2 = pushPredicateIntoCase((RexCall) exp);
+        if (exp2 != exp) {
+          expList.set(i, exp2);
+        }
+      }
+    }
+
     // Find reducible expressions.
     final List<RexNode> constExps = Lists.newArrayList();
     List<Boolean> addCasts = Lists.newArrayList();
@@ -466,6 +479,45 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     return ImmutableMap.copyOf(builder);
   }
 
+  private static RexCall pushPredicateIntoCase(RexCall call) {
+    if (call.getType().getSqlTypeName() != SqlTypeName.BOOLEAN) {
+      return call;
+    }
+    int caseOrdinal = -1;
+    final List<RexNode> operands = call.getOperands();
+    for (int i = 0; i < operands.size(); i++) {
+      RexNode operand = operands.get(i);
+      switch (operand.getKind()) {
+      case CASE:
+        caseOrdinal = i;
+      }
+    }
+    if (caseOrdinal < 0) {
+      return call;
+    }
+    // Convert
+    //   f(CASE WHEN p1 THEN v1 ... END, arg)
+    // to
+    //   CASE WHEN p1 THEN f(v1, arg) ... END
+    final RexCall case_ = (RexCall) operands.get(caseOrdinal);
+    final List<RexNode> nodes = new ArrayList<>();
+    for (int i = 0; i < case_.getOperands().size(); i++) {
+      RexNode node = case_.getOperands().get(i);
+      if (!RexUtil.isCasePredicate(case_, i)) {
+        node = substitute(call, caseOrdinal, node);
+      }
+      nodes.add(node);
+    }
+    return case_.clone(call.getType(), nodes);
+  }
+
+  /** Converts op(arg0, ..., argOrdinal, ..., argN) to op(arg0,..., node, ..., argN). */
+  private static RexNode substitute(RexCall call, int ordinal, RexNode node) {
+    final List<RexNode> newOperands = Lists.newArrayList(call.getOperands());
+    newOperands.set(ordinal, node);
+    return call.clone(call.getType(), newOperands);
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**
@@ -499,8 +551,12 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
 
     @Override public RexNode visitCall(RexCall call) {
       RexNode node = visit(call);
-      if (node == null) {
-        return super.visitCall(call);
+      if (node != null) {
+        return node;
+      }
+      node = super.visitCall(call);
+      if (node != call) {
+        node = RexUtil.simplify(rexBuilder, node);
       }
       return node;
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/21cf1259/core/src/main/java/org/apache/calcite/rex/RexCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexCall.java b/core/src/main/java/org/apache/calcite/rex/RexCall.java
index 0300f54..b06ffc9 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCall.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCall.java
@@ -117,6 +117,8 @@ public class RexCall extends RexNode {
     switch (getKind()) {
     case IS_NOT_NULL:
       return !operands.get(0).getType().isNullable();
+    case CAST:
+      return operands.get(0).isAlwaysTrue();
     default:
       return false;
     }
@@ -126,6 +128,8 @@ public class RexCall extends RexNode {
     switch (getKind()) {
     case IS_NULL:
       return !operands.get(0).getType().isNullable();
+    case CAST:
+      return operands.get(0).isAlwaysFalse();
     default:
       return false;
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/21cf1259/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
index f0d5c3d..bc89347 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
@@ -23,9 +23,6 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Sets;
-
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -304,7 +301,7 @@ public class RexProgramBuilder {
    *              sub-expression exists.
    */
   private RexLocalRef registerInternal(RexNode expr, boolean force) {
-    expr = simplify(expr);
+    expr = RexUtil.simplify(rexBuilder, expr);
 
     RexLocalRef ref;
     final Pair<String, String> key;
@@ -343,27 +340,6 @@ public class RexProgramBuilder {
     }
   }
 
-  /** Simplifies AND(x, x) into x, and similar. */
-  private static RexNode simplify(RexNode node) {
-    switch (node.getKind()) {
-    case AND:
-    case OR:
-      // Convert:
-      //   AND(x, x) into x
-      //   OR(x, y, x) into OR(x, y)
-      final RexCall call = (RexCall) node;
-      if (!Util.isDistinct(call.getOperands())) {
-        final List<RexNode> list2 =
-            ImmutableList.copyOf(Sets.newLinkedHashSet(call.getOperands()));
-        if (list2.size() == 1) {
-          return list2.get(0);
-        }
-        return new RexCall(call.getType(), call.getOperator(), list2);
-      }
-    }
-    return node;
-  }
-
   /**
    * Adds an expression to the list of common expressions, and returns a
    * reference to the expression. <b>DOES NOT CHECK WHETHER THE EXPRESSION

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/21cf1259/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 d470a4f..ee384e1 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.rex;
 
+import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.function.Predicate1;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelCollation;
@@ -37,6 +38,7 @@ import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.base.Function;
+import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
@@ -44,8 +46,10 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -126,7 +130,7 @@ public class RexUtil {
     int n = fieldList.size();
     assert n == lhsRowType.getFieldCount()
         : "field count: lhs [" + lhsRowType + "] rhs [" + rhsRowType + "]";
-    List<RexNode> rhsExps = new ArrayList<RexNode>();
+    List<RexNode> rhsExps = new ArrayList<>();
     for (RelDataTypeField field : fieldList) {
       rhsExps.add(
           rexBuilder.makeInputRef(field.getType(), field.getIndex()));
@@ -147,7 +151,7 @@ public class RexUtil {
       RelDataType lhsRowType,
       List<RexNode> rhsExps) {
     List<RelDataTypeField> lhsFields = lhsRowType.getFieldList();
-    List<RexNode> castExps = new ArrayList<RexNode>();
+    List<RexNode> castExps = new ArrayList<>();
     for (Pair<RelDataTypeField, RexNode> pair
         : Pair.zip(lhsFields, rhsExps, true)) {
       RelDataTypeField lhsField = pair.left;
@@ -748,15 +752,13 @@ public class RexUtil {
   public static List<RelCollation> apply(
       Mappings.TargetMapping mapping,
       List<RelCollation> collationList) {
-    final List<RelCollation> newCollationList =
-        new ArrayList<RelCollation>();
+    final List<RelCollation> newCollationList = new ArrayList<>();
     for (RelCollation collation : collationList) {
       if (collation == RelCollations.PRESERVE) {
         newCollationList.add(collation);
         continue;
       }
-      final List<RelFieldCollation> newFieldCollationList =
-          new ArrayList<RelFieldCollation>();
+      final List<RelFieldCollation> newFieldCollationList = new ArrayList<>();
       for (RelFieldCollation fieldCollation
           : collation.getFieldCollations()) {
         final RelFieldCollation newFieldCollation =
@@ -834,8 +836,7 @@ public class RexUtil {
   public static List<RelFieldCollation> applyFields(
       Mappings.TargetMapping mapping,
       List<RelFieldCollation> fieldCollations) {
-    final List<RelFieldCollation> newFieldCollations =
-        new ArrayList<RelFieldCollation>(fieldCollations.size());
+    final List<RelFieldCollation> newFieldCollations = new ArrayList<>();
     for (RelFieldCollation fieldCollation : fieldCollations) {
       newFieldCollations.add(apply(mapping, fieldCollation));
     }
@@ -855,8 +856,8 @@ public class RexUtil {
   public static Iterable<RexNode> apply(Mappings.TargetMapping mapping,
       Iterable<? extends RexNode> nodes) {
     final RexPermuteInputsShuttle shuttle = RexPermuteInputsShuttle.of(mapping);
-    return Iterables.transform(nodes,
-        new Function<RexNode, RexNode>() {
+    return Iterables.transform(
+        nodes, new Function<RexNode, RexNode>() {
           public RexNode apply(RexNode input) {
             return input.accept(shuttle);
           }
@@ -947,7 +948,7 @@ public class RexUtil {
       //noinspection unchecked
       return (List) exprs;
     }
-    final List<RexNode> list = new ArrayList<RexNode>();
+    final List<RexNode> list = new ArrayList<>();
     flattenRecurse(list, exprs, op);
     return list;
   }
@@ -1156,14 +1157,283 @@ public class RexUtil {
     return Lists.transform(types, FAMILY_FN);
   }
 
+  /** Removes all expressions from a list that are equivalent to a given
+   * expression. Returns whether any were removed. */
+  public static boolean removeAll(List<RexNode> targets, RexNode e) {
+    int count = 0;
+    Iterator<RexNode> iterator = targets.iterator();
+    while (iterator.hasNext()) {
+      RexNode next = iterator.next();
+      if (equivalent(next, e)) {
+        ++count;
+        iterator.remove();
+      }
+    }
+    return count > 0;
+  }
+
+  /** Returns whether two expressions are equivalent. */
+  private static boolean equivalent(RexNode e1, RexNode e2) {
+    // TODO: make broader;
+    // 1. 'x = y' should be equivalent to 'y = x'.
+    // 2. 'c2 and c1' should be equivalent to 'c1 and c2'.
+    return e1 == e2 || e1.toString().equals(e2.toString());
+  }
+
+  /**
+   * Simplifies a boolean expression.
+   *
+   * <p>In particular:</p>
+   * <ul>
+   * <li>{@code simplify(x = 1 AND y = 2 AND NOT x = 1)}
+   * returns {@code y = 2}</li>
+   * <li>{@code simplify(x = 1 AND FALSE)}
+   * returns {@code FALSE}</li>
+   * </ul>
+   */
+  public static RexNode simplify(RexBuilder rexBuilder, RexNode e) {
+    switch (e.getKind()) {
+    case AND:
+      return simplifyAnd(rexBuilder, (RexCall) e);
+    case OR:
+      return simplifyOr(rexBuilder, (RexCall) e);
+    case CASE:
+      return simplifyCase(rexBuilder, (RexCall) e);
+    case IS_NULL:
+      return ((RexCall) e).getOperands().get(0).getType().isNullable()
+          ? e : rexBuilder.makeLiteral(false);
+    case IS_NOT_NULL:
+      return ((RexCall) e).getOperands().get(0).getType().isNullable()
+          ? e : rexBuilder.makeLiteral(true);
+    default:
+      return e;
+    }
+  }
+
+  private static RexNode simplifyCase(RexBuilder rexBuilder, RexCall call) {
+    final List<RexNode> operands = call.getOperands();
+    final List<RexNode> newOperands = new ArrayList<>();
+    for (int i = 0; i < operands.size(); i++) {
+      RexNode operand = operands.get(i);
+      if (isCasePredicate(call, i)) {
+        if (operand.isAlwaysTrue()) {
+          // Predicate is always TRUE. Make value the ELSE and quit.
+          newOperands.add(operands.get(i + 1));
+          break;
+        }
+        if (operand.isAlwaysFalse()) {
+          // Predicate is always FALSE. Skip predicate and value.
+          ++i;
+          continue;
+        }
+      }
+      newOperands.add(operand);
+    }
+    assert newOperands.size() % 2 == 1;
+    switch (newOperands.size()) {
+    case 1:
+      return newOperands.get(0);
+    }
+  trueFalse:
+    if (call.getType().getSqlTypeName() == SqlTypeName.BOOLEAN) {
+      // Optimize CASE where every branch returns constant true or constant
+      // false:
+      //   CASE
+      //   WHEN p1 THEN TRUE
+      //   WHEN p2 THEN FALSE
+      //   WHEN p3 THEN TRUE
+      //   ELSE FALSE
+      //   END
+      final List<Pair<RexNode, RexNode>> pairs =
+          casePairs(rexBuilder, newOperands);
+      for (Ord<Pair<RexNode, RexNode>> pair : Ord.zip(pairs)) {
+        if (!pair.e.getValue().isAlwaysTrue()
+            && !pair.e.getValue().isAlwaysFalse()) {
+          break trueFalse;
+        }
+      }
+      final List<RexNode> terms = new ArrayList<>();
+      final List<RexNode> notTerms = new ArrayList<>();
+      for (Ord<Pair<RexNode, RexNode>> pair : Ord.zip(pairs)) {
+        if (pair.e.getValue().isAlwaysTrue()) {
+          terms.add(andNot(rexBuilder, pair.e.getKey(), notTerms));
+        } else {
+          notTerms.add(pair.e.getKey());
+        }
+      }
+      return composeDisjunction(rexBuilder, terms, false);
+    }
+    if (newOperands.equals(operands)) {
+      return call;
+    }
+    return call.clone(call.getType(), newOperands);
+  }
+
+  /** Given "CASE WHEN p1 THEN v1 ... ELSE e END"
+   * returns [(p1, v1), ..., (true, e)]. */
+  private static List<Pair<RexNode, RexNode>> casePairs(RexBuilder rexBuilder,
+      List<RexNode> operands) {
+    final ImmutableList.Builder<Pair<RexNode, RexNode>> builder =
+        ImmutableList.builder();
+    for (int i = 0; i < operands.size() - 1; i += 2) {
+      builder.add(Pair.of(operands.get(i), operands.get(i + 1)));
+    }
+    builder.add(
+        Pair.of((RexNode) rexBuilder.makeLiteral(true), Util.last(operands)));
+    return builder.build();
+  }
+
+  public static RexNode simplifyAnd(RexBuilder rexBuilder, RexCall e) {
+    final List<RexNode> terms = RelOptUtil.conjunctions(e);
+    final List<RexNode> notTerms = new ArrayList<>();
+    for (int i = 0; i < terms.size(); i++) {
+      final RexNode term = terms.get(i);
+      switch (term.getKind()) {
+      case NOT:
+        notTerms.add(
+            ((RexCall) term).getOperands().get(0));
+        terms.remove(i);
+        --i;
+        break;
+      case LITERAL:
+        if (!RexLiteral.booleanValue(term)) {
+          return term; // false
+        } else {
+          terms.remove(i);
+          --i;
+        }
+      }
+    }
+    if (terms.isEmpty() && notTerms.isEmpty()) {
+      return rexBuilder.makeLiteral(true);
+    }
+    // If one of the not-disjunctions is a disjunction that is wholly
+    // contained in the disjunctions list, the expression is not
+    // satisfiable.
+    //
+    // Example #1. x AND y AND z AND NOT (x AND y)  - not satisfiable
+    // Example #2. x AND y AND NOT (x AND y)        - not satisfiable
+    // Example #3. x AND y AND NOT (x AND y AND z)  - may be satisfiable
+    for (RexNode notDisjunction : notTerms) {
+      final List<RexNode> terms2 = RelOptUtil.conjunctions(notDisjunction);
+      if (terms.containsAll(terms2)) {
+        return rexBuilder.makeLiteral(false);
+      }
+    }
+    // Add the NOT disjunctions back in.
+    for (RexNode notDisjunction : notTerms) {
+      terms.add(
+          rexBuilder.makeCall(
+              SqlStdOperatorTable.NOT, notDisjunction));
+    }
+    return composeConjunction(rexBuilder, terms, false);
+  }
+
+  /** Simplifies OR(x, x) into x, and similar. */
+  public static RexNode simplifyOr(RexBuilder rexBuilder, RexCall call) {
+    assert call.getKind() == SqlKind.OR;
+    final List<RexNode> terms = RelOptUtil.disjunctions(call);
+    for (int i = 0; i < terms.size(); i++) {
+      final RexNode term = terms.get(i);
+      switch (term.getKind()) {
+      case LITERAL:
+        if (RexLiteral.booleanValue(term)) {
+          return term; // true
+        } else {
+          terms.remove(i);
+          --i;
+        }
+      }
+    }
+    return composeDisjunction(rexBuilder, terms, false);
+  }
+
+  /**
+   * Creates the expression {@code e1 AND NOT notTerm1 AND NOT notTerm2 ...}.
+   */
+  public static RexNode andNot(RexBuilder rexBuilder, RexNode e,
+      RexNode... notTerms) {
+    return andNot(rexBuilder, e, Arrays.asList(notTerms));
+  }
+
+  /**
+   * Creates the expression {@code e1 AND NOT notTerm1 AND NOT notTerm2 ...}.
+   *
+   * <p>Examples:
+   * <ul>
+   *   <li>andNot(p) returns "p"
+   *   <li>andNot(p, n1, n2) returns "p AND NOT n1 AND NOT n2"
+   *   <li>andNot(x = 10, x = 20, y = 30, x = 30)
+   *       returns "x = 10 AND NOT (y = 30)"
+   * </ul>
+   */
+  public static RexNode andNot(final RexBuilder rexBuilder, RexNode e,
+      Iterable<? extends RexNode> notTerms) {
+    // If "e" is of the form "x = literal", remove all "x = otherLiteral"
+    // terms from notTerms.
+    switch (e.getKind()) {
+    case EQUALS:
+      final RexCall call = (RexCall) e;
+      if (call.getOperands().get(1) instanceof RexLiteral) {
+        notTerms = Iterables.filter(
+            notTerms, new Predicate<RexNode>() {
+              public boolean apply(RexNode input) {
+                switch (input.getKind()) {
+                case EQUALS:
+                  RexCall call2 = (RexCall) input;
+                  if (call2.getOperands().get(0)
+                      .equals(call.getOperands().get(0))
+                      && call2.getOperands().get(1) instanceof RexLiteral) {
+                    return false;
+                  }
+                }
+                return true;
+              }
+            });
+      }
+    }
+    return composeConjunction(
+        rexBuilder, Iterables.concat(
+            ImmutableList.of(e), Iterables.transform(
+                notTerms, notFn(rexBuilder))), false);
+  }
+
+  /** Returns whether a given operand of a CASE expression is a predicate.
+   *
+   * <p>A switched case (CASE x WHEN x1 THEN v1 ... ELSE e END) has an even
+   * number of arguments and odd-numbered arguments are predicates.
+   *
+   * <p>A condition case (CASE WHEN p1 THEN v1 ... ELSE e END) has an odd
+   * number of arguments and even-numbered arguments are predicates, except for
+   * the last argument. */
+  public static boolean isCasePredicate(RexCall call, int i) {
+    assert call.getKind() == SqlKind.CASE;
+    return i < call.operands.size() - 1
+        && (call.operands.size() - i) % 2 == 1;
+  }
+
+  /** Returns a function that applies NOT to its argument. */
+  public static Function<RexNode, RexNode> notFn(final RexBuilder rexBuilder) {
+    return new Function<RexNode, RexNode>() {
+      public RexNode apply(RexNode input) {
+        return input.isAlwaysTrue()
+            ? rexBuilder.makeLiteral(false)
+            : input.isAlwaysFalse()
+            ? rexBuilder.makeLiteral(true)
+            : input.getKind() == SqlKind.NOT
+            ? ((RexCall) input).operands.get(0)
+            : rexBuilder.makeCall(SqlStdOperatorTable.NOT, input);
+      }
+    };
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**
    * Walks over expressions and builds a bank of common sub-expressions.
    */
   private static class ExpressionNormalizer extends RexVisitorImpl<RexNode> {
-    final Map<String, RexNode> mapDigestToExpr =
-        new HashMap<String, RexNode>();
+    final Map<String, RexNode> mapDigestToExpr = new HashMap<>();
     final boolean allowDups;
 
     protected ExpressionNormalizer(boolean allowDups) {
@@ -1197,7 +1467,7 @@ public class RexUtil {
     }
 
     public RexNode visitCall(RexCall call) {
-      List<RexNode> normalizedOperands = new ArrayList<RexNode>();
+      List<RexNode> normalizedOperands = new ArrayList<>();
       int diffCount = 0;
       for (RexNode operand : call.getOperands()) {
         operand.accept(this);
@@ -1296,7 +1566,7 @@ public class RexUtil {
 
     public FieldAccessFinder() {
       super(true);
-      fieldAccessList = new ArrayList<RexFieldAccess>();
+      fieldAccessList = new ArrayList<>();
     }
 
     public Void visitFieldAccess(RexFieldAccess fieldAccess) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/21cf1259/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
index 931002b..a8402d6 100644
--- a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
@@ -26,6 +26,7 @@ import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 
 import com.google.common.base.Function;
@@ -287,7 +288,7 @@ public class MaterializationTest {
 
   /** Unit test for logic functions
    * {@link org.apache.calcite.plan.SubstitutionVisitor#mayBeSatisfiable} and
-   * {@link org.apache.calcite.plan.SubstitutionVisitor#simplify}. */
+   * {@link RexUtil#simplify}. */
   @Test public void testSatisfiable() {
     // TRUE may be satisfiable
     checkSatisfiable(rexBuilder.makeLiteral(true), "true");
@@ -441,13 +442,13 @@ public class MaterializationTest {
 
   private void checkNotSatisfiable(RexNode e) {
     assertFalse(SubstitutionVisitor.mayBeSatisfiable(e));
-    final RexNode simple = SubstitutionVisitor.simplify(rexBuilder, e);
+    final RexNode simple = RexUtil.simplify(rexBuilder, e);
     assertFalse(RexLiteral.booleanValue(simple));
   }
 
   private void checkSatisfiable(RexNode e, String s) {
     assertTrue(SubstitutionVisitor.mayBeSatisfiable(e));
-    final RexNode simple = SubstitutionVisitor.simplify(rexBuilder, e);
+    final RexNode simple = RexUtil.simplify(rexBuilder, e);
     assertEquals(s, simple.toString());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/21cf1259/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 059ab57..8a62e89 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -723,6 +723,71 @@ public class RelOptRulesTest extends RelOptTestBase {
         "select p1 is not distinct from p0 from (values (2, cast(null as integer))) as t(p0, p1)");
   }
 
+  // see HIVE-9645
+  @Test public void testReduceConstantsNullEqualsOne() throws Exception {
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(ReduceExpressionsRule.PROJECT_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.JOIN_INSTANCE)
+        .build();
+
+    checkPlanning(program,
+        "select count(1) from emp where cast(null as integer) = 1");
+  }
+
+  // see HIVE-9644
+  @Test public void testReduceConstantsCaseEquals() throws Exception {
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(ReduceExpressionsRule.PROJECT_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.JOIN_INSTANCE)
+        .build();
+
+    // Equivalent to 'deptno = 10'
+    checkPlanning(program,
+        "select count(1) from emp\n"
+            + "where case deptno\n"
+            + "  when 20 then 2\n"
+            + "  when 10 then 1\n"
+            + "  else 3 end = 1");
+  }
+
+  @Test public void testReduceConstantsCaseEquals2() throws Exception {
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(ReduceExpressionsRule.PROJECT_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.JOIN_INSTANCE)
+        .build();
+
+    // Equivalent to 'case when deptno = 20 then false
+    //                     when deptno = 10 then true
+    //                     else null end'
+    checkPlanning(program,
+        "select count(1) from emp\n"
+            + "where case deptno\n"
+            + "  when 20 then 2\n"
+            + "  when 10 then 1\n"
+            + "  else cast(null as integer) end = 1");
+  }
+
+  @Test public void testReduceConstantsCaseEquals3() throws Exception {
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(ReduceExpressionsRule.PROJECT_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.JOIN_INSTANCE)
+        .build();
+
+    // Equivalent to 'deptno = 30 or deptno = 10'
+    checkPlanning(program,
+        "select count(1) from emp\n"
+            + "where case deptno\n"
+            + "  when 30 then 1\n"
+            + "  when 20 then 2\n"
+            + "  when 10 then 1\n"
+            + "  when 30 then 111\n"
+            + "  else 0 end = 1");
+  }
+
   @Test public void testReduceConstantsEliminatesFilter() throws Exception {
     HepProgram program = new HepProgramBuilder()
         .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/21cf1259/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 d7eecc4..b06d203 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -89,6 +89,11 @@ public class RexProgramTest {
         equalTo(expected));
   }
 
+  private void checkSimplify(RexNode node, String expected) {
+    assertThat(RexUtil.simplify(rexBuilder, node).toString(),
+        equalTo(expected));
+  }
+
   /** Returns the number of nodes (including leaves) in a Rex tree. */
   private static int nodeCount(RexNode node) {
     int n = 1;
@@ -124,6 +129,10 @@ public class RexProgramTest {
         ImmutableList.copyOf(nodes));
   }
 
+  private RexNode case_(RexNode... nodes) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.CASE, nodes);
+  }
+
   private RexNode eq(RexNode n1, RexNode n2) {
     return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, n1, n2);
   }
@@ -633,6 +642,82 @@ public class RexProgramTest {
                             or(fRef,
                                and(gRef, or(trueLiteral, falseLiteral)))))))));
   }
+
+  @Test public void testSimplify() {
+    final RelDataType booleanType =
+        typeFactory.createSqlType(SqlTypeName.BOOLEAN);
+    final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
+    final RelDataType rowType = typeFactory.builder()
+        .add("a", booleanType)
+        .add("b", booleanType)
+        .add("c", booleanType)
+        .add("d", booleanType)
+        .add("e", booleanType)
+        .add("f", booleanType)
+        .add("g", booleanType)
+        .add("h", intType)
+        .build();
+
+    final RexDynamicParam range = rexBuilder.makeDynamicParam(rowType, 0);
+    final RexNode aRef = rexBuilder.makeFieldAccess(range, 0);
+    final RexNode bRef = rexBuilder.makeFieldAccess(range, 1);
+    final RexNode cRef = rexBuilder.makeFieldAccess(range, 2);
+    final RexNode dRef = rexBuilder.makeFieldAccess(range, 3);
+    final RexNode eRef = rexBuilder.makeFieldAccess(range, 4);
+    final RexLiteral true_ = rexBuilder.makeLiteral(true);
+    final RexLiteral false_ = rexBuilder.makeLiteral(false);
+
+    // and: remove duplicates
+    checkSimplify(and(aRef, bRef, aRef), "AND(?0.a, ?0.b)");
+
+    // and: remove true
+    checkSimplify(and(aRef, bRef, true_),
+        "AND(?0.a, ?0.b)");
+
+    // and: false falsifies
+    checkSimplify(and(aRef, bRef, false_),
+        "false");
+
+    // or: remove duplicates
+    checkSimplify(or(aRef, bRef, aRef), "OR(?0.a, ?0.b)");
+
+    // or: remove false
+    checkSimplify(or(aRef, bRef, false_),
+        "OR(?0.a, ?0.b)");
+
+    // or: true makes everything true
+    checkSimplify(or(aRef, bRef, true_), "true");
+
+    // case: remove false branches
+    checkSimplify(case_(eq(bRef, cRef), dRef, false_, aRef, eRef),
+        "CASE(=(?0.b, ?0.c), ?0.d, ?0.e)");
+
+    // case: true branches become the last branch
+    checkSimplify(
+        case_(eq(bRef, cRef), dRef, true_, aRef, eq(cRef, dRef), eRef, cRef),
+        "CASE(=(?0.b, ?0.c), ?0.d, ?0.a)");
+
+    // case: singleton
+    checkSimplify(case_(true_, aRef, eq(cRef, dRef), eRef, cRef), "?0.a");
+
+    // case: form an AND of branches that return true
+    checkSimplify(
+        case_(aRef, true_, bRef, false_, cRef, false_, dRef, true_, false_),
+        "OR(?0.a, AND(?0.d, NOT(?0.b), NOT(?0.c)))");
+
+    checkSimplify(
+        case_(aRef, true_, bRef, false_, cRef, false_, dRef, true_, eRef,
+            false_, true_),
+        "OR(?0.a, AND(?0.d, NOT(?0.b), NOT(?0.c)), AND(NOT(?0.b), NOT(?0.c), NOT(?0.e)))");
+
+    // is null, applied to not-null value
+    checkSimplify(rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, aRef),
+        "false");
+
+    // is not null, applied to not-null value
+    checkSimplify(rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, aRef),
+        "true");
+  }
 }
 
 // End RexProgramTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/21cf1259/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index 6623c90..1baaac4 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -540,7 +540,7 @@ LogicalProject(EXPR$0=[+(1, 2)], EXPR$1=[+($0, +(3, 4))], EXPR$2=[+(+(5, 6), $0)
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EXPR$0=[3], EXPR$1=[+($0, 7)], EXPR$2=[+(11, $0)], EXPR$3=[null], EXPR$4=[CAST(2):INTEGER], EXPR$5=[ROW(15)])
-  LogicalFilter(condition=[AND(=($0, 15), =($0, 15), =($0, CAST(2):INTEGER))])
+  LogicalFilter(condition=[AND(=($0, 15), =($0, CAST(2):INTEGER))])
     LogicalProject(DEPTNO=[$0], NAME=[$1], EMPNO=[$2], ENAME=[$3], JOB=[$4], MGR=[$5], HIREDATE=[$6], SAL=[$7], COMM=[$8], DEPTNO0=[$9], SLACKER=[$10])
       LogicalJoin(condition=[=($0, $11)], joinType=[inner])
         LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
@@ -829,7 +829,7 @@ LogicalProject(EXPR$0=[CAST(CASE(IS NULL($1), IS NULL($0), IS NULL($0), IS NULL(
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(EXPR$0=[CAST(CASE(IS NULL($1), IS NULL($0), IS NULL($0), IS NULL($1), =($1, $0))):BOOLEAN NOT NULL])
+LogicalProject(EXPR$0=[CASE(IS NULL($1), IS NULL($0), CAST(=($1, $0)):BOOLEAN NOT NULL)])
   LogicalProject(EXPR$0=[2], EXPR$1=[null])
     LogicalValues(tuples=[[{ 0 }]])
 ]]>
@@ -3656,4 +3656,101 @@ LogicalProject(DEPTNO=[$0], DEPTNO0=[$9])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testReduceConstantsNullEqualsOne">
+        <Resource name="sql">
+            <![CDATA[select count(1) from emp where cast(null as integer) = 1]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[1])
+    LogicalFilter(condition=[=(null, 1)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[1])
+    LogicalValues(tuples=[[]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testReduceConstantsCaseEquals">
+        <Resource name="sql">
+            <![CDATA[select count(1) from emp
+where case deptno
+  when 20 then 2
+  when 10 then 1
+  else 3 end = 1]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[1])
+    LogicalFilter(condition=[=(CASE(=($7, 20), 2, =($7, 10), 1, 3), 1)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[1])
+    LogicalFilter(condition=[=($7, 10)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testReduceConstantsCaseEquals2">
+        <Resource name="sql">
+            <![CDATA[select count(1) from emp
+where case deptno
+  when 20 then 2
+  when 10 then 1
+  else cast(null as integer) end = 1]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[1])
+    LogicalFilter(condition=[=(CASE(=($7, 20), 2, =($7, 10), 1, null), 1)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[1])
+    LogicalFilter(condition=[CASE(=($7, 20), CAST(false):BOOLEAN, =($7, 10), CAST(true):BOOLEAN, null)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testReduceConstantsCaseEquals3">
+        <Resource name="sql">
+            <![CDATA[select count(1) from emp
+where case deptno
+  when 30 then 1
+  when 20 then 2
+  when 10 then 1
+  when 30 then 111
+  else 0 end = 1]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[1])
+    LogicalFilter(condition=[=(CASE(=($7, 30), 1, =($7, 20), 2, =($7, 10), 1, =($7, 30), 111, 0), 1)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[1])
+    LogicalFilter(condition=[OR(=($7, 30), =($7, 10))])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
 </Root>