You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2018/05/26 05:32:06 UTC

calcite git commit: [CALCITE-2247] Simplify AND and OR conditions using predicates (Zoltan Haindrich)

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


[CALCITE-2247] Simplify AND and OR conditions using predicates (Zoltan Haindrich)

If you are simplifying "p1 AND p2", when you are simplifying "p2" you
know that "p1" is not true, and so can add it to the predicates.

Close apache/calcite#688


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

Branch: refs/heads/master
Commit: c3788056507206c50b8c700ab81abd4b511bb863
Parents: f17c376
Author: Zoltan Haindrich <ki...@rxd.hu>
Authored: Mon May 7 14:59:30 2018 +0200
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri May 25 13:40:52 2018 -0700

----------------------------------------------------------------------
 .../org/apache/calcite/rex/RexSimplify.java     | 142 ++++++++++++-
 .../apache/calcite/test/RelOptRulesTest.java    |  22 ++
 .../org/apache/calcite/test/RexProgramTest.java | 207 ++++++++++++++++++-
 .../org/apache/calcite/test/RelOptRulesTest.xml |  67 +++++-
 4 files changed, 419 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/c3788056/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
index fc1a524..7befc4d 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -268,6 +268,55 @@ public class RexSimplify {
     }
   }
 
+  private void simplifyAndTerms(List<RexNode> terms) {
+    RexSimplify simplify = withUnknownAsFalse(false);
+    for (int i = 0; i < terms.size(); i++) {
+      RexNode t = terms.get(i);
+      if (Predicate.of(t) == null) {
+        continue;
+      }
+      terms.set(i, simplify.simplify(t));
+      RelOptPredicateList newPredicates = simplify.predicates.union(rexBuilder,
+          RelOptPredicateList.of(rexBuilder, terms.subList(i, i + 1)));
+      simplify = simplify.withPredicates(newPredicates);
+    }
+    for (int i = 0; i < terms.size(); i++) {
+      RexNode t = terms.get(i);
+      if (Predicate.of(t) != null) {
+        continue;
+      }
+      terms.set(i, simplify.simplify(t));
+    }
+  }
+
+  private void simplifyOrTerms(List<RexNode> terms) {
+    // Suppose we are processing "e1(x) OR e2(x) OR e3(x)". When we are
+    // visiting "e3(x)" we know both "e1(x)" and "e2(x)" are not true (they
+    // may be unknown), because if either of them were true we would have
+    // stopped.
+    RexSimplify simplify = this;
+    for (int i = 0; i < terms.size(); i++) {
+      final RexNode t = terms.get(i);
+      if (Predicate.of(t) == null) {
+        continue;
+      }
+      final RexNode t2 = simplify.simplify(t);
+      terms.set(i, t2);
+      final RexNode inverse =
+          simplify.simplify(rexBuilder.makeCall(SqlStdOperatorTable.NOT, t2));
+      final RelOptPredicateList newPredicates = simplify.predicates.union(rexBuilder,
+          RelOptPredicateList.of(rexBuilder, ImmutableList.of(inverse)));
+      simplify = simplify.withPredicates(newPredicates);
+    }
+    for (int i = 0; i < terms.size(); i++) {
+      final RexNode t = terms.get(i);
+      if (Predicate.of(t) != null) {
+        continue;
+      }
+      terms.set(i, simplify.simplify(t));
+    }
+  }
+
   private RexNode simplifyNot(RexCall call) {
     final RexNode a = call.getOperands().get(0);
     switch (a.getKind()) {
@@ -312,6 +361,12 @@ public class RexSimplify {
   private RexNode simplifyIs(RexCall call) {
     final SqlKind kind = call.getKind();
     final RexNode a = call.getOperands().get(0);
+
+    final RexNode pred = simplifyIsPredicate(kind, a);
+    if (pred != null) {
+      return pred;
+    }
+
     final RexNode simplified = simplifyIs2(kind, a);
     if (simplified != null) {
       return simplified;
@@ -319,6 +374,25 @@ public class RexSimplify {
     return call;
   }
 
+  private RexNode simplifyIsPredicate(SqlKind kind, RexNode a) {
+    if (!RexUtil.isReferenceOrAccess(a, true)) {
+      return null;
+    }
+
+    for (RexNode p : predicates.pulledUpPredicates) {
+      IsPredicate pred = IsPredicate.of(p);
+      if (pred == null || !a.toString().equals(pred.ref.toString())) {
+        continue;
+      }
+      if (kind == pred.kind) {
+        return rexBuilder.makeLiteral(true);
+      } else {
+        return rexBuilder.makeLiteral(false);
+      }
+    }
+    return null;
+  }
+
   private RexNode simplifyIs2(SqlKind kind, RexNode a) {
     switch (kind) {
     case IS_NULL:
@@ -374,6 +448,9 @@ public class RexSimplify {
     if (!a.getType().isNullable()) {
       return rexBuilder.makeLiteral(true);
     }
+    if (predicates.pulledUpPredicates.contains(a)) {
+      return rexBuilder.makeLiteral(true);
+    }
     switch (Strong.policy(a.getKind())) {
     case ANY:
       final List<RexNode> operands = new ArrayList<>();
@@ -564,8 +641,15 @@ public class RexSimplify {
     final List<RexNode> terms = new ArrayList<>();
     final List<RexNode> notTerms = new ArrayList<>();
     RelOptUtil.decomposeConjunction(e, terms, notTerms);
-    simplifyList(terms);
+
+    if (unknownAsFalse) {
+      simplifyAndTerms(terms);
+    } else {
+      simplifyList(terms);
+    }
+
     simplifyList(notTerms);
+
     if (unknownAsFalse) {
       return simplifyAnd2ForUnknownAsFalse(terms, notTerms);
     }
@@ -614,7 +698,7 @@ public class RexSimplify {
   private <C extends Comparable<C>> RexNode simplifyAnd2ForUnknownAsFalse(
       List<RexNode> terms, List<RexNode> notTerms, Class<C> clazz) {
     for (RexNode term : terms) {
-      if (term.isAlwaysFalse()) {
+      if (term.isAlwaysFalse() || RexLiteral.isNullLiteral(term)) {
         return rexBuilder.makeLiteral(false);
       }
     }
@@ -855,8 +939,10 @@ public class RexSimplify {
       // no change
       return e;
     } else if (range2.equals(Range.all())) {
-      // Term is always satisfied given these predicates
-      return rexBuilder.makeLiteral(true);
+      // Range is always satisfied given these predicates; but nullability might
+      // be problematic
+      return simplify(
+          rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, comparison.ref));
     } else if (range2.lowerEndpoint().equals(range2.upperEndpoint())) {
       if (range2.lowerBoundType() == BoundType.OPEN
           || range2.upperBoundType() == BoundType.OPEN) {
@@ -922,6 +1008,7 @@ public class RexSimplify {
   public RexNode simplifyOr(RexCall call) {
     assert call.getKind() == SqlKind.OR;
     final List<RexNode> terms = RelOptUtil.disjunctions(call);
+    simplifyOrTerms(terms);
     return simplifyOrs(terms);
   }
 
@@ -932,7 +1019,13 @@ public class RexSimplify {
       final RexNode term = simplify(terms.get(i));
       switch (term.getKind()) {
       case LITERAL:
-        if (!RexLiteral.isNullLiteral(term)) {
+        if (RexLiteral.isNullLiteral(term)) {
+          if (unknownAsFalse) {
+            terms.remove(i);
+            --i;
+            continue;
+          }
+        } else {
           if (RexLiteral.booleanValue(term)) {
             return term; // true
           } else {
@@ -1306,9 +1399,21 @@ public class RexSimplify {
     }
   }
 
+  /** Marker interface for predicates (expressions that evaluate to BOOLEAN). */
+  private interface Predicate {
+    /** Wraps an expression in a Predicate or returns null. */
+    static Predicate of(RexNode t) {
+      final Predicate p = Comparison.of(t);
+      if (p != null) {
+        return p;
+      }
+      return IsPredicate.of(t);
+    }
+  }
+
   /** Comparison between a {@link RexInputRef} or {@link RexFieldAccess} and a
    * literal. Literal may be on left or right side, and may be null. */
-  private static class Comparison {
+  private static class Comparison implements Predicate {
     final RexNode ref;
     final SqlKind kind;
     final RexLiteral literal;
@@ -1349,6 +1454,31 @@ public class RexSimplify {
     }
   }
 
+  /** Represents an IS Predicate. */
+  private static class IsPredicate implements Predicate {
+    final RexNode ref;
+    final SqlKind kind;
+
+    private IsPredicate(RexNode ref, SqlKind kind) {
+      this.ref = Preconditions.checkNotNull(ref);
+      this.kind = Preconditions.checkNotNull(kind);
+    }
+
+    /** Creates an IS predicate, or returns null. */
+    static IsPredicate of(RexNode e) {
+      switch (e.getKind()) {
+      case IS_NULL:
+      case IS_NOT_NULL:
+        RexNode pA = ((RexCall) e).getOperands().get(0);
+        if (!RexUtil.isReferenceOrAccess(pA, true)) {
+          return null;
+        }
+        return new IsPredicate(pA, e.getKind());
+      }
+      return null;
+    }
+  }
+
   private static boolean isUpperBound(final RexNode e) {
     final List<RexNode> operands;
     switch (e.getKind()) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/c3788056/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 1ab37c2..028e7eb 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -1571,6 +1571,28 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkPlanning(program, sql);
   }
 
+  @Test public void testOrAlwaysTrue() {
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(ReduceExpressionsRule.PROJECT_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.JOIN_INSTANCE)
+        .build();
+    final String sql = "select * from EMPNULLABLES_20\n"
+        + "where sal is null or sal is not null";
+    checkPlanning(program, sql);
+  }
+
+  @Test public void testOrAlwaysTrue2() {
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(ReduceExpressionsRule.PROJECT_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.JOIN_INSTANCE)
+        .build();
+    final String sql = "select * from EMPNULLABLES_20\n"
+        + "where sal is not null or sal is null";
+    checkPlanning(program, sql);
+  }
+
   @Test public void testReduceConstants2() throws Exception {
     HepProgram program = new HepProgramBuilder()
         .addRuleInstance(ReduceExpressionsRule.PROJECT_INSTANCE)

http://git-wip-us.apache.org/repos/asf/calcite/blob/c3788056/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 449dd97..90b56bc 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -64,6 +64,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Multimap;
 
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.math.BigDecimal;
@@ -1211,8 +1212,9 @@ public class RexProgramTest {
         "AND(<=(?0.a, 1), >(?0.a, 1))",
         "false");
 
-    checkSimplify(and(le(aRef, literal1), ge(aRef, literal1)),
-        "AND(<=(?0.a, 1), >=(?0.a, 1))");
+    checkSimplify2(and(le(aRef, literal1), ge(aRef, literal1)),
+        "AND(<=(?0.a, 1), >=(?0.a, 1))",
+        "=(?0.a, 1)");
 
     checkSimplify2(and(lt(aRef, literal1), eq(aRef, literal1), ge(aRef, literal1)),
         "AND(<(?0.a, 1), =(?0.a, 1), >=(?0.a, 1))",
@@ -1312,7 +1314,7 @@ public class RexProgramTest {
         "false");
 
     checkSimplifyFilter(and(le(aRef, literal1), ge(aRef, literal1)),
-        "AND(<=(?0.a, 1), >=(?0.a, 1))");
+        "=(?0.a, 1)");
 
     checkSimplifyFilter(and(lt(aRef, literal1), eq(aRef, literal1), ge(aRef, literal1)),
         "false");
@@ -1472,6 +1474,205 @@ public class RexProgramTest {
         "false");
   }
 
+  @Test public void testSimplifyAndPush() {
+    final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
+    final RelDataType rowType = typeFactory.builder()
+        .add("a", intType)
+        .add("b", intType)
+        .build();
+
+    final RexDynamicParam range = rexBuilder.makeDynamicParam(rowType, 0);
+    final RexNode aRef = rexBuilder.makeFieldAccess(range, 0);
+    final RexNode bRef = rexBuilder.makeFieldAccess(range, 1);
+    final RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
+    final RexLiteral literal5 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(5));
+    final RexLiteral literal10 = rexBuilder.makeExactLiteral(BigDecimal.TEN);
+
+    checkSimplifyFilter(
+        or(
+            or(eq(aRef, literal1),
+                eq(aRef, literal1)),
+            eq(aRef, literal1)),
+        "=(?0.a, 1)");
+
+    checkSimplifyFilter(
+        or(
+            and(eq(aRef, literal1),
+                eq(aRef, literal1)),
+            and(eq(aRef, literal10),
+                eq(aRef, literal1))),
+        "=(?0.a, 1)");
+
+    checkSimplifyFilter(
+        and(
+            eq(aRef, literal1),
+            or(eq(aRef, literal1),
+                eq(aRef, literal10))),
+        "=(?0.a, 1)");
+    checkSimplifyFilter(
+        and(
+            or(eq(aRef, literal1),
+                eq(aRef, literal10)),
+            eq(aRef, literal1)),
+        "=(?0.a, 1)");
+
+    checkSimplifyFilter(
+        and(gt(aRef, literal10),
+            gt(aRef, literal1)),
+        ">(?0.a, 10)");
+
+    checkSimplifyFilter(
+        and(gt(aRef, literal1),
+            gt(aRef, literal10)),
+        ">(?0.a, 10)");
+  }
+
+  @Test public void testSimplifyOrTerms() {
+    final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
+    final RelDataType rowType = typeFactory.builder()
+        .add("a", intType).nullable(false)
+        .add("b", intType).nullable(true)
+        .add("c", intType).nullable(true)
+        .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 RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
+    final RexLiteral literal2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(2));
+    final RexLiteral literal3 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(3));
+    final RexLiteral literal4 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(4));
+
+    // "a != 1 or a = 1" ==> "true"
+    checkSimplifyFilter(
+        or(ne(aRef, literal1),
+            eq(aRef, literal1)),
+        "true");
+
+    // TODO: make this simplify to "true"
+    checkSimplifyFilter(
+        or(eq(aRef, literal1),
+            ne(aRef, literal1)),
+        "OR(=(?0.a, 1), <>(?0.a, 1))");
+
+    // "b != 1 or b = 1" cannot be simplified, because b might be null
+    final RexNode neOrEq =
+        or(ne(bRef, literal1),
+            eq(bRef, literal1));
+    checkSimplifyFilter(neOrEq, "OR(<>(?0.b, 1), IS NOT NULL(?0.b))");
+
+    // Careful of the excluded middle!
+    // We cannot simplify "b != 1 or b = 1" to "true" because if b is null, the
+    // result is unknown.
+    // TODO: "b is not unknown" would be the best simplification.
+    assertThat(simplify.withUnknownAsFalse(false).simplify(neOrEq).toString(),
+        equalTo("OR(<>(?0.b, 1), IS NOT NULL(?0.b))"));
+
+    // "a is null or a is not null" ==> "true"
+    checkSimplifyFilter(
+        or(isNull(aRef),
+            isNotNull(aRef)),
+        "true");
+
+    // "a is not null or a is null" ==> "true"
+    checkSimplifyFilter(
+        or(isNotNull(aRef),
+            isNull(aRef)),
+        "true");
+
+    // "b is not null or b is null" ==> "true" (valid even though b nullable)
+    checkSimplifyFilter(
+        or(isNotNull(bRef),
+            isNull(bRef)),
+        "true");
+
+    // "b is not null or c is null" unchanged
+    checkSimplifyFilter(
+        or(isNotNull(bRef),
+            isNull(cRef)),
+        "OR(IS NOT NULL(?0.b), IS NULL(?0.c))");
+
+    // multiple predicates are handled correctly
+    checkSimplifyFilter(
+        and(
+            or(eq(bRef, literal1),
+                eq(bRef, literal2)),
+            eq(bRef, literal2),
+            eq(aRef, literal3),
+            or(eq(aRef, literal3),
+                eq(aRef, literal4))),
+        "AND(=(?0.b, 2), =(?0.a, 3))");
+  }
+
+  @Test public void testSimplifyUnknown() {
+    final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
+    final RelDataType rowType = typeFactory.builder()
+        .add("a", intType).nullable(true)
+        .build();
+
+    final RexDynamicParam range = rexBuilder.makeDynamicParam(rowType, 0);
+    final RexNode aRef = rexBuilder.makeFieldAccess(range, 0);
+    final RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
+
+
+    checkSimplify2(
+        and(eq(aRef, literal1),
+            nullLiteral),
+        "AND(=(?0.a, 1), null)",
+        "false");
+    checkSimplify2(
+        and(trueLiteral,
+            nullLiteral),
+        "null",
+        "false");
+    checkSimplify2(
+        and(falseLiteral,
+            nullLiteral),
+        "false",
+        "false");
+
+    checkSimplify2(
+        and(nullLiteral,
+            eq(aRef, literal1)),
+        "AND(null, =(?0.a, 1))",
+        "false");
+
+    checkSimplify2(
+        or(eq(aRef, literal1),
+            nullLiteral),
+        "OR(=(?0.a, 1), null)",
+        "=(?0.a, 1)");
+    checkSimplify2(
+        or(trueLiteral,
+            nullLiteral),
+        "true",
+        "true");
+    checkSimplify2(
+        or(falseLiteral,
+            nullLiteral),
+        "null",
+        "false");
+  }
+
+  @Ignore
+  @Test public void testSimplifyAnd3() {
+    final RelDataType boolType = typeFactory.createSqlType(SqlTypeName.BOOLEAN);
+    final RelDataType rowType = typeFactory.builder()
+        .add("a", boolType).nullable(true)
+        .build();
+
+    final RexDynamicParam range = rexBuilder.makeDynamicParam(rowType, 0);
+    final RexNode aRef = rexBuilder.makeFieldAccess(range, 0);
+
+    // in the case of 3-valued logic, the result must be unknown if a is unknown
+    checkSimplify2(
+        and(aRef,
+            not(aRef)),
+        "a is null and null",
+        "false");
+  }
+
   /** Unit test for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-1289">[CALCITE-1289]
    * RexUtil.simplifyCase() should account for nullability</a>. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/c3788056/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 272b0e6..af39d3c 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -3076,6 +3076,52 @@ LogicalProject(EXPR$0=[CAST(/($0, $1)):INTEGER NOT NULL])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testOrAlwaysTrue">
+        <Resource name="sql">
+            <![CDATA[select * from EMPNULLABLES_20
+where sal is null or sal is not null]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$7])
+  LogicalFilter(condition=[OR(IS NULL($5), IS NOT NULL($5))])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$8])
+      LogicalFilter(condition=[AND(=($7, 20), >($5, 1000))])
+        LogicalTableScan(table=[[CATALOG, SALES, EMPNULLABLES]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$7])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$8])
+    LogicalFilter(condition=[AND(=($7, 20), >($5, 1000))])
+      LogicalTableScan(table=[[CATALOG, SALES, EMPNULLABLES]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testOrAlwaysTrue2">
+        <Resource name="sql">
+            <![CDATA[select * from EMPNULLABLES_20
+where sal is not null or sal is null]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$7])
+  LogicalFilter(condition=[OR(IS NOT NULL($5), IS NULL($5))])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$8])
+      LogicalFilter(condition=[AND(=($7, 20), >($5, 1000))])
+        LogicalTableScan(table=[[CATALOG, SALES, EMPNULLABLES]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$7])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$8])
+    LogicalFilter(condition=[AND(=($7, 20), >($5, 1000))])
+      LogicalTableScan(table=[[CATALOG, SALES, EMPNULLABLES]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testWindowFunctionOnAggregations">
         <Resource name="sql">
             <![CDATA[SELECT
@@ -4495,14 +4541,15 @@ LogicalProject(EXPR$0=[1])
             <![CDATA[
 LogicalProject(EXPR$0=[1])
   LogicalJoin(condition=[=($0, $8)], joinType=[inner])
-    LogicalUnion(all=[true])
-      LogicalProject(DEPTNO=[$7])
-        LogicalFilter(condition=[>($7, 7)])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-      LogicalProject(DEPTNO=[$7])
-        LogicalFilter(condition=[>($7, 10)])
-          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-    LogicalFilter(condition=[OR(>($7, 7), >($7, 10))])
+    LogicalFilter(condition=[>($0, 7)])
+      LogicalUnion(all=[true])
+        LogicalProject(DEPTNO=[$7])
+          LogicalFilter(condition=[>($7, 7)])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalProject(DEPTNO=[$7])
+          LogicalFilter(condition=[>($7, 10)])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalFilter(condition=[>($7, 7)])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -4822,7 +4869,7 @@ LogicalProject(EXPR$0=[1])
       LogicalProject(DEPTNO=[$7])
         LogicalFilter(condition=[>($7, 1)])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-    LogicalFilter(condition=[OR(>($7, 7), >($7, 10), >($7, 1))])
+    LogicalFilter(condition=[OR(>($7, 7), >($7, 1))])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -7558,7 +7605,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EMPNO=[10], ENAME=[$1], JOB=[$2], MGR=[null], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[7], SLACKER=[$8])
-  LogicalFilter(condition=[AND(=($7, 7), IS NULL($3), =($0, 10))])
+  LogicalFilter(condition=[AND(=($7, 7), =($0, 10), IS NULL($3))])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>