You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by kg...@apache.org on 2019/03/05 21:50:14 UTC

[calcite] branch master updated: [CALCITE-2886] Simplification of AND expressions should push negations earlier

This is an automated email from the ASF dual-hosted git repository.

kgyrtkirk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git


The following commit(s) were added to refs/heads/master by this push:
     new 36d373f  [CALCITE-2886] Simplification of AND expressions should push negations earlier
36d373f is described below

commit 36d373f0ff549eb079ed91c7cf2dda2475a0721a
Author: Zoltan Haindrich <ki...@rxd.hu>
AuthorDate: Mon Mar 4 14:58:49 2019 +0100

    [CALCITE-2886] Simplification of AND expressions should push negations earlier
    
    Earlier decomposition of AND operands into normal/negated terms preceeded recursion of simplification;
    which could have helped retaining negated terms.
    Remove duplicate invocation of simplification on negated and operands.
    Also run full simplification on filter predicates.
---
 .../java/org/apache/calcite/rex/RexSimplify.java   | 33 +++++++-------
 .../java/org/apache/calcite/runtime/FlatLists.java |  2 +-
 .../apache/calcite/test/MaterializationTest.java   |  5 +--
 .../org/apache/calcite/test/RexProgramTest.java    | 38 +++++++++-------
 .../org/apache/calcite/test/RelOptRulesTest.xml    | 18 ++++----
 core/src/test/resources/sql/blank.iq               | 13 +++---
 core/src/test/resources/sql/sub-query.iq           | 50 +++++++++++-----------
 7 files changed, 82 insertions(+), 77 deletions(-)

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 e00f46f..ab68011 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -347,7 +347,7 @@ public class RexSimplify {
     }
 
     if (o0.getType().getSqlTypeName() == SqlTypeName.BOOLEAN) {
-      Comparison cmp = Comparison.of(e, node -> true);
+      Comparison cmp = Comparison.of(rexBuilder.makeCall(e.getOperator(), o0, o1), node -> true);
       if (cmp != null) {
         if (cmp.literal.isAlwaysTrue()) {
           switch (cmp.kind) {
@@ -356,7 +356,7 @@ public class RexSimplify {
             return cmp.ref;
           case LESS_THAN:
           case NOT_EQUALS: // x!=true
-            return rexBuilder.makeCall(SqlStdOperatorTable.NOT, cmp.ref);
+            return simplify(rexBuilder.makeCall(SqlStdOperatorTable.NOT, cmp.ref), unknownAs);
           case GREATER_THAN:
             /* this is false, but could be null if x is null */
             if (!cmp.ref.getType().isNullable()) {
@@ -375,7 +375,7 @@ public class RexSimplify {
           switch (cmp.kind) {
           case EQUALS:
           case LESS_THAN_OR_EQUAL:
-            return rexBuilder.makeCall(SqlStdOperatorTable.NOT, cmp.ref);
+            return simplify(rexBuilder.makeCall(SqlStdOperatorTable.NOT, cmp.ref), unknownAs);
           case NOT_EQUALS:
           case GREATER_THAN:
             return cmp.ref;
@@ -468,7 +468,6 @@ public class RexSimplify {
   }
 
   private void simplifyList(List<RexNode> terms, RexUnknownAs unknownAs) {
-    unknownAs = UNKNOWN; // TODO
     for (int i = 0; i < terms.size(); i++) {
       terms.set(i, simplify(terms.get(i), unknownAs));
     }
@@ -1151,17 +1150,20 @@ public class RexSimplify {
   }
 
   RexNode simplifyAnd(RexCall e, RexUnknownAs unknownAs) {
-    final List<RexNode> terms = new ArrayList<>();
-    final List<RexNode> notTerms = new ArrayList<>();
-    RelOptUtil.decomposeConjunction(e, terms, notTerms);
+    List<RexNode> operands = RelOptUtil.conjunctions(e);
 
     if (unknownAs == FALSE && predicateElimination) {
-      simplifyAndTerms(terms, FALSE);
+      simplifyAndTerms(operands, FALSE);
     } else {
-      simplifyList(terms, unknownAs);
+      simplifyList(operands, unknownAs);
     }
 
-    simplifyList(notTerms, unknownAs.negate());
+    final List<RexNode> terms = new ArrayList<>();
+    final List<RexNode> notTerms = new ArrayList<>();
+
+    for (RexNode o : operands) {
+      RelOptUtil.decomposeConjunction(o, terms, notTerms);
+    }
 
     switch (unknownAs) {
     case FALSE:
@@ -1449,9 +1451,7 @@ public class RexSimplify {
     }
     // Add the NOT disjunctions back in.
     for (RexNode notDisjunction : notTerms) {
-      final RexNode call =
-          rexBuilder.makeCall(SqlStdOperatorTable.NOT, notDisjunction);
-      terms.add(simplify(call, FALSE));
+      terms.add(rexBuilder.makeCall(SqlStdOperatorTable.NOT, notDisjunction));
     }
     // The negated terms: only deterministic expressions
     for (RexNode negatedTerm : negatedTerms) {
@@ -1562,6 +1562,8 @@ public class RexSimplify {
     final List<RexNode> terms = RelOptUtil.disjunctions(call);
     if (predicateElimination) {
       simplifyOrTerms(terms, unknownAs);
+    } else {
+      simplifyList(terms, unknownAs);
     }
     return simplifyOrs(terms, unknownAs);
   }
@@ -1585,7 +1587,7 @@ public class RexSimplify {
    * Modifies the list in place. */
   private RexNode simplifyOrs(List<RexNode> terms, RexUnknownAs unknownAs) {
     for (int i = 0; i < terms.size(); i++) {
-      final RexNode term = simplify(terms.get(i), unknownAs);
+      final RexNode term = terms.get(i);
       switch (term.getKind()) {
       case LITERAL:
         if (RexLiteral.isNullLiteral(term)) {
@@ -2160,7 +2162,8 @@ public class RexSimplify {
   public RexNode simplifyFilterPredicates(Iterable<? extends RexNode> predicates) {
     final RexNode simplifiedAnds =
         withPredicateElimination(Bug.CALCITE_2401_FIXED)
-            .simplifyAnds(predicates, FALSE);
+            .simplifyUnknownAsFalse(
+                RexUtil.composeConjunction(rexBuilder, predicates));
     if (simplifiedAnds.isAlwaysFalse()) {
       return null;
     }
diff --git a/core/src/main/java/org/apache/calcite/runtime/FlatLists.java b/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
index d491e14..a2418f0 100644
--- a/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
+++ b/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
@@ -140,7 +140,7 @@ public class FlatLists {
    * @param t Array of members of list
    * @return List containing the given members
    */
-  private static <T extends Object & Comparable> ComparableList<T> flatList_(
+  private static <T extends Comparable> ComparableList<T> flatList_(
       T[] t, boolean copy) {
     switch (t.length) {
     case 0:
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 3126db2..fd767de 100644
--- a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
@@ -37,7 +37,6 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexSimplify;
-import org.apache.calcite.rex.RexUnknownAs;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.schema.QueryableTable;
@@ -759,13 +758,13 @@ public class MaterializationTest {
 
   private void checkNotSatisfiable(RexNode e) {
     assertFalse(SubstitutionVisitor.mayBeSatisfiable(e));
-    final RexNode simple = simplify.simplifyUnknownAs(e, RexUnknownAs.UNKNOWN);
+    final RexNode simple = simplify.simplifyUnknownAsFalse(e);
     assertFalse(RexLiteral.booleanValue(simple));
   }
 
   private void checkSatisfiable(RexNode e, String s) {
     assertTrue(SubstitutionVisitor.mayBeSatisfiable(e));
-    final RexNode simple = simplify.simplifyUnknownAs(e, RexUnknownAs.UNKNOWN);
+    final RexNode simple = simplify.simplifyUnknownAsFalse(e);
     assertEquals(s, simple.toString());
   }
 
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 e7b4ac4..7581d97 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -1241,19 +1241,19 @@ public class RexProgramTest extends RexProgramBuilderBase {
     // "x != x" simplifies to "false"
     checkSimplify(ne(literal1, literal1), "false");
     checkSimplify(ne(hRef, hRef), "false");
-    checkSimplify2(ne(iRef, iRef), "AND(null, IS NULL(?0.i))", "false");
+    checkSimplify3(ne(iRef, iRef), "AND(null, IS NULL(?0.i))", "false", "IS NULL(?0.i)");
     checkSimplifyUnchanged(ne(iRef, hRef));
 
     // "x < x" simplifies to "false"
     checkSimplify(lt(literal1, literal1), "false");
     checkSimplify(lt(hRef, hRef), "false");
-    checkSimplify2(lt(iRef, iRef), "AND(null, IS NULL(?0.i))", "false");
+    checkSimplify3(lt(iRef, iRef), "AND(null, IS NULL(?0.i))", "false", "IS NULL(?0.i)");
     checkSimplifyUnchanged(lt(iRef, hRef));
 
     // "x > x" simplifies to "false"
     checkSimplify(gt(literal1, literal1), "false");
     checkSimplify(gt(hRef, hRef), "false");
-    checkSimplify2(gt(iRef, iRef), "AND(null, IS NULL(?0.i))", "false");
+    checkSimplify3(gt(iRef, iRef), "AND(null, IS NULL(?0.i))", "false", "IS NULL(?0.i)");
     checkSimplifyUnchanged(gt(iRef, hRef));
 
     // "(not x) is null" to "x is null"
@@ -1546,11 +1546,12 @@ public class RexProgramTest extends RexProgramBuilderBase {
         ">(?0.a, 10)");
 
     // "null AND NOT(null OR x)" => "null AND NOT(x)"
-    checkSimplify2(
+    checkSimplify3(
         and(nullBool,
             not(or(nullBool, vBool()))),
         "AND(null, NOT(?0.bool0))",
-        "false");
+        "false",
+        "NOT(?0.bool0)");
 
     // "x1 AND x2 AND x3 AND NOT(x1) AND NOT(x2) AND NOT(x0)" =>
     // "x3 AND null AND x1 IS NULL AND x2 IS NULL AND NOT(x0)"
@@ -1683,21 +1684,23 @@ public class RexProgramTest extends RexProgramBuilderBase {
             nullInt),
         "AND(=(?0.a, 1), null:INTEGER)",
         "false");
-    checkSimplify2(
+    checkSimplify3(
         and(trueLiteral,
             nullBool),
         "null:BOOLEAN",
-        "false");
+        "false",
+        "true");
     checkSimplify(
         and(falseLiteral,
             nullBool),
         "false");
 
-    checkSimplify2(
+    checkSimplify3(
         and(nullBool,
             eq(aRef, literal1)),
         "AND(null, =(?0.a, 1))",
-        "false");
+        "false",
+        "=(?0.a, 1)");
 
     checkSimplify3(
         or(eq(aRef, literal1),
@@ -2269,12 +2272,14 @@ public class RexProgramTest extends RexProgramBuilderBase {
    * <a href="https://issues.apache.org/jira/browse/CALCITE-2421">[CALCITE-2421]
    * to-be-filled </a>. */
   @Test public void testSelfComparisions() {
-    checkSimplify2(and(eq(vInt(), vInt()), eq(vInt(1), vInt(1))),
+    checkSimplify3(and(eq(vInt(), vInt()), eq(vInt(1), vInt(1))),
         "AND(OR(null, IS NOT NULL(?0.int0)), OR(null, IS NOT NULL(?0.int1)))",
-        "AND(IS NOT NULL(?0.int0), IS NOT NULL(?0.int1))");
-    checkSimplify2(and(ne(vInt(), vInt()), ne(vInt(1), vInt(1))),
+        "AND(IS NOT NULL(?0.int0), IS NOT NULL(?0.int1))",
+        "true");
+    checkSimplify3(and(ne(vInt(), vInt()), ne(vInt(1), vInt(1))),
         "AND(null, IS NULL(?0.int0), IS NULL(?0.int1))",
-        "false");
+        "false",
+        "AND(IS NULL(?0.int0), IS NULL(?0.int1))");
   }
 
   @Test public void testBooleanComparisions() {
@@ -2494,9 +2499,10 @@ public class RexProgramTest extends RexProgramBuilderBase {
     // "x = x AND NOT (y >= y)"
     //    -> "x = x AND y < y" (treating unknown as unknown)
     //    -> false (treating unknown as false)
-    checkSimplify2(and(eq(vInt(1), vInt(1)), not(ge(vInt(2), vInt(2)))),
+    checkSimplify3(and(eq(vInt(1), vInt(1)), not(ge(vInt(2), vInt(2)))),
         "AND(OR(null, IS NOT NULL(?0.int1)), null, IS NULL(?0.int2))",
-        "false");
+        "false",
+        "IS NULL(?0.int2)");
 
     // "NOT(x = x AND NOT (y = y))"
     //   -> "OR(x <> x, y >= y)" (treating unknown as unknown)
@@ -2526,7 +2532,7 @@ public class RexProgramTest extends RexProgramBuilderBase {
     checkSimplify3(not(or(eq(vInt(1), vInt(1)), not(ge(vInt(2), vInt(2))))),
         "AND(null, IS NULL(?0.int1), OR(null, IS NOT NULL(?0.int2)))",
         "false",
-        "AND(null, IS NULL(?0.int1))");
+        "IS NULL(?0.int1)");
   }
 
   private RexNode simplify(RexNode e) {
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 c94842c..92dc2b1 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -481,7 +481,7 @@ LogicalProject(DEPTNO=[$0])
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    LogicalFilter(condition=[AND(OR(IS NOT TRUE(<=($0, $9)), =($10, 0)), OR(<=($10, $11), =($10, 0), <=($0, $9)), OR(>($0, $9), =($10, 0), <=($0, $9), >($10, $11)))])
+    LogicalFilter(condition=[OR(=($10, 0), AND(>($0, $9), <>($10, 0), IS NOT TRUE(<=($0, $9)), <=($10, $11)))])
       LogicalJoin(condition=[true], joinType=[inner])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
         LogicalAggregate(group=[{}], m=[MAX($0)], c=[COUNT()], d=[COUNT($0)])
@@ -493,7 +493,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    LogicalFilter(condition=[AND(OR(IS NOT TRUE(<=($0, $9)), =($10, 0)), OR(<=($10, $11), =($10, 0), <=($0, $9)), OR(>($0, $9), =($10, 0), <=($0, $9), >($10, $11)))])
+    LogicalFilter(condition=[OR(=($10, 0), AND(>($0, $9), <>($10, 0), IS NOT TRUE(<=($0, $9)), <=($10, $11)))])
       LogicalJoin(condition=[true], joinType=[inner])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
         LogicalAggregate(group=[{}], m=[MAX($0)], c=[COUNT()], d=[COUNT($0)])
@@ -7135,7 +7135,7 @@ LogicalProject(DEPTNO=[$0])
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    LogicalFilter(condition=[OR(AND(IS TRUE(>($0, $9)), <>($10, 0)), AND(>($0, $9), <>($10, 0), IS NOT TRUE(>($0, $9)), <=($10, $11)))])
+    LogicalFilter(condition=[OR(AND(>($0, $9), <>($10, 0)), AND(>($0, $9), <>($10, 0), IS NOT TRUE(>($0, $9)), <=($10, $11)))])
       LogicalJoin(condition=[true], joinType=[inner])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
         LogicalAggregate(group=[{}], m=[MIN($0)], c=[COUNT()], d=[COUNT($0)])
@@ -7147,7 +7147,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    LogicalFilter(condition=[OR(AND(IS TRUE(>($0, $9)), <>($10, 0)), AND(>($0, $9), <>($10, 0), IS NOT TRUE(>($0, $9)), <=($10, $11)))])
+    LogicalFilter(condition=[OR(AND(>($0, $9), <>($10, 0)), AND(>($0, $9), <>($10, 0), IS NOT TRUE(>($0, $9)), <=($10, $11)))])
       LogicalJoin(condition=[true], joinType=[inner])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
         LogicalAggregate(group=[{}], m=[MIN($0)], c=[COUNT()], d=[COUNT($0)])
@@ -8085,7 +8085,7 @@ LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null:INTEGER)])
             <![CDATA[
 LogicalProject(EMPNO=[$0])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    LogicalFilter(condition=[<($0, CASE(AND(IS NOT NULL($12), <>($9, 0)), 10, OR(=($9, 0), AND(<>($9, 0), IS NULL($12), >=($10, $9))), 20, 30))])
+    LogicalFilter(condition=[<($0, CASE(OR(AND(IS NOT NULL($12), <>($9, 0)), AND(<($10, $9), null, <>($9, 0), IS NULL($12))), 10, AND(OR(IS NULL($12), =($9, 0)), OR(>=($10, $9), =($9, 0), IS NOT NULL($12))), 20, 30))])
       LogicalJoin(condition=[=($7, $11)], joinType=[left])
         LogicalJoin(condition=[true], joinType=[inner])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
@@ -8512,7 +8512,7 @@ LogicalProject(DEPTNO=[$0])
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(SAL=[$5])
-  LogicalFilter(condition=[OR(IS NOT TRUE(OR(IS NOT NULL($13), <($11, $10))), =($10, 0))])
+  LogicalFilter(condition=[OR(=($10, 0), IS NOT TRUE(OR(IS NOT NULL($13), <($11, $10))))])
     LogicalJoin(condition=[AND(=($0, $12), =($2, $14))], joinType=[left])
       LogicalJoin(condition=[=($2, $9)], joinType=[left])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
@@ -8527,7 +8527,7 @@ LogicalProject(SAL=[$5])
             <![CDATA[
 LogicalProject(SAL=[$5])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    LogicalFilter(condition=[OR(IS NOT TRUE(OR(IS NOT NULL($12), <($10, $9))), =($9, 0))])
+    LogicalFilter(condition=[OR(=($9, 0), IS NOT TRUE(OR(IS NOT NULL($12), <($10, $9))))])
       LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{2}])
         LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{2}])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
@@ -8604,7 +8604,7 @@ LogicalProject(EMPNO=[$1])
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    LogicalFilter(condition=[OR(IS NOT TRUE(OR(IS NOT NULL($12), <($10, $9))), =($9, 0))])
+    LogicalFilter(condition=[OR(=($9, 0), IS NOT TRUE(OR(IS NOT NULL($12), <($10, $9))))])
       LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{1}])
         LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{1}])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
@@ -8623,7 +8623,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  LogicalFilter(condition=[OR(IS NOT TRUE(OR(IS NOT NULL($13), <($11, $10))), =($10, 0))])
+  LogicalFilter(condition=[OR(=($10, 0), IS NOT TRUE(OR(IS NOT NULL($13), <($11, $10))))])
     LogicalJoin(condition=[AND(=($0, $12), =($1, $14))], joinType=[left])
       LogicalJoin(condition=[=($1, $9)], joinType=[left])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
diff --git a/core/src/test/resources/sql/blank.iq b/core/src/test/resources/sql/blank.iq
index 1cae199..3af2287 100644
--- a/core/src/test/resources/sql/blank.iq
+++ b/core/src/test/resources/sql/blank.iq
@@ -73,14 +73,13 @@ insert into table2 values (NULL, 1), (2, 1);
 # Checked on Oracle
 !set lateDecorrelate true
 select i, j from table1 where table1.j NOT IN (select i from table2 where table1.i=table2.j);
-EnumerableCalc(expr#0..7=[{inputs}], expr#8=[IS NOT NULL($t7)], expr#9=[<($t4, $t3)], expr#10=[OR($t8, $t9)], expr#11=[IS NOT TRUE($t10)], expr#12=[0], expr#13=[=($t3, $t12)], expr#14=[IS NULL($t1)], expr#15=[OR($t11, $t13, $t14)], proj#0..1=[{exprs}], $condition=[$t15])
+EnumerableCalc(expr#0..7=[{inputs}], expr#8=[0], expr#9=[=($t3, $t8)], expr#10=[IS NOT TRUE($t9)], expr#11=[IS NOT NULL($t7)], expr#12=[<($t4, $t3)], expr#13=[OR($t11, $t12)], expr#14=[IS NOT TRUE($t13)], expr#15=[IS NOT NULL($t1)], expr#16=[AND($t10, $t14, $t15)], expr#17=[OR($t9, $t16)], proj#0..1=[{exprs}], $condition=[$t17])
   EnumerableJoin(condition=[AND(=($0, $6), =($1, $5))], joinType=[left])
-    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NOT NULL($t1)], expr#6=[0], expr#7=[=($t3, $t6)], expr#8=[OR($t5, $t7)], proj#0..4=[{exprs}], $condition=[$t8])
-      EnumerableJoin(condition=[=($0, $2)], joinType=[left])
-        EnumerableTableScan(table=[[BLANK, TABLE1]])
-        EnumerableAggregate(group=[{1}], c=[COUNT()], ck=[COUNT($0)])
-          EnumerableCalc(expr#0..1=[{inputs}], expr#2=[IS NOT NULL($t1)], proj#0..1=[{exprs}], $condition=[$t2])
-            EnumerableTableScan(table=[[BLANK, TABLE2]])
+    EnumerableJoin(condition=[=($0, $2)], joinType=[left])
+      EnumerableTableScan(table=[[BLANK, TABLE1]])
+      EnumerableAggregate(group=[{1}], c=[COUNT()], ck=[COUNT($0)])
+        EnumerableCalc(expr#0..1=[{inputs}], expr#2=[IS NOT NULL($t1)], proj#0..1=[{exprs}], $condition=[$t2])
+          EnumerableTableScan(table=[[BLANK, TABLE2]])
     EnumerableCalc(expr#0..1=[{inputs}], expr#2=[true], proj#0..2=[{exprs}])
       EnumerableAggregate(group=[{0, 1}])
         EnumerableCalc(expr#0..1=[{inputs}], expr#2=[IS NOT NULL($t1)], expr#3=[IS NOT NULL($t0)], expr#4=[AND($t2, $t3)], proj#0..1=[{exprs}], $condition=[$t4])
diff --git a/core/src/test/resources/sql/sub-query.iq b/core/src/test/resources/sql/sub-query.iq
index ebe5e87..3282fd0 100644
--- a/core/src/test/resources/sql/sub-query.iq
+++ b/core/src/test/resources/sql/sub-query.iq
@@ -32,23 +32,22 @@ where t1.x not in (select t2.x from t2);
 (0 rows)
 
 !ok
-EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NULL($t4)], expr#6=[>=($t2, $t1)], expr#7=[AND($t5, $t6)], expr#8=[0], expr#9=[=($t1, $t8)], expr#10=[IS NULL($t0)], expr#11=[OR($t7, $t9, $t10)], X=[$t0], $condition=[$t11])
+EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t1, $t5)], expr#7=[<>($t1, $t5)], expr#8=[IS NULL($t4)], expr#9=[>=($t2, $t1)], expr#10=[IS NOT NULL($t0)], expr#11=[AND($t7, $t8, $t9, $t10)], expr#12=[OR($t6, $t11)], X=[$t0], $condition=[$t12])
   EnumerableJoin(condition=[=($0, $3)], joinType=[left])
-    EnumerableCalc(expr#0..2=[{inputs}], expr#3=[IS NOT NULL($t0)], expr#4=[0], expr#5=[=($t1, $t4)], expr#6=[OR($t3, $t5)], proj#0..2=[{exprs}], $condition=[$t6])
-      EnumerableJoin(condition=[true], joinType=[inner])
+    EnumerableJoin(condition=[true], joinType=[inner])
+      EnumerableUnion(all=[true])
+        EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
+          EnumerableValues(tuples=[[{ 0 }]])
+        EnumerableCalc(expr#0=[{inputs}], expr#1=[2], EXPR$0=[$t1])
+          EnumerableValues(tuples=[[{ 0 }]])
+        EnumerableCalc(expr#0=[{inputs}], expr#1=[null:INTEGER], EXPR$0=[$t1])
+          EnumerableValues(tuples=[[{ 0 }]])
+      EnumerableAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)])
         EnumerableUnion(all=[true])
           EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
             EnumerableValues(tuples=[[{ 0 }]])
-          EnumerableCalc(expr#0=[{inputs}], expr#1=[2], EXPR$0=[$t1])
-            EnumerableValues(tuples=[[{ 0 }]])
           EnumerableCalc(expr#0=[{inputs}], expr#1=[null:INTEGER], EXPR$0=[$t1])
             EnumerableValues(tuples=[[{ 0 }]])
-        EnumerableAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)])
-          EnumerableUnion(all=[true])
-            EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
-              EnumerableValues(tuples=[[{ 0 }]])
-            EnumerableCalc(expr#0=[{inputs}], expr#1=[null:INTEGER], EXPR$0=[$t1])
-              EnumerableValues(tuples=[[{ 0 }]])
     EnumerableAggregate(group=[{0, 1}])
       EnumerableCalc(expr#0=[{inputs}], expr#1=[true], proj#0..1=[{exprs}])
         EnumerableUnion(all=[true])
@@ -1440,7 +1439,7 @@ select sal from "scott".emp
 (0 rows)
 
 !ok
-EnumerableCalc(expr#0..3=[{inputs}], expr#4=[IS NOT FALSE($t2)], expr#5=[IS NULL($t3)], expr#6=[OR($t4, $t5)], expr#7=[NOT($t2)], expr#8=[OR($t5, $t7)], expr#9=[AND($t6, $t8)], SAL=[$t1], $condition=[$t9])
+EnumerableCalc(expr#0..3=[{inputs}], expr#4=[IS NULL($t3)], SAL=[$t1], $condition=[$t4])
   EnumerableJoin(condition=[true], joinType=[left])
     EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], SAL=[$t5])
       EnumerableTableScan(table=[[scott, EMP]])
@@ -1461,7 +1460,7 @@ select sal from "scott".emp
 (0 rows)
 
 !ok
-EnumerableCalc(expr#0..3=[{inputs}], expr#4=[NOT($t2)], expr#5=[IS NOT NULL($t2)], expr#6=[OR($t4, $t5)], expr#7=[IS NOT TRUE($t6)], expr#8=[IS NULL($t3)], expr#9=[OR($t7, $t8)], SAL=[$t1], $condition=[$t9])
+EnumerableCalc(expr#0..3=[{inputs}], expr#4=[IS NULL($t3)], expr#5=[NOT($t2)], expr#6=[IS NOT NULL($t2)], expr#7=[OR($t5, $t6)], expr#8=[IS NOT TRUE($t7)], expr#9=[OR($t4, $t8)], SAL=[$t1], $condition=[$t9])
   EnumerableJoin(condition=[true], joinType=[left])
     EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], SAL=[$t5])
       EnumerableTableScan(table=[[scott, EMP]])
@@ -1482,7 +1481,7 @@ select sal from "scott".emp
 (0 rows)
 
 !ok
-EnumerableCalc(expr#0..3=[{inputs}], expr#4=[IS NOT FALSE($t2)], expr#5=[IS NULL($t3)], expr#6=[OR($t4, $t5)], expr#7=[NOT($t2)], expr#8=[OR($t5, $t7)], expr#9=[AND($t6, $t8)], SAL=[$t1], $condition=[$t9])
+EnumerableCalc(expr#0..3=[{inputs}], expr#4=[IS NULL($t3)], SAL=[$t1], $condition=[$t4])
   EnumerableJoin(condition=[true], joinType=[left])
     EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], SAL=[$t5])
       EnumerableTableScan(table=[[scott, EMP]])
@@ -1503,7 +1502,7 @@ select sal from "scott".emp
 (0 rows)
 
 !ok
-EnumerableCalc(expr#0..3=[{inputs}], expr#4=[IS NOT FALSE($t2)], expr#5=[IS NULL($t3)], expr#6=[OR($t4, $t5)], expr#7=[NOT($t2)], expr#8=[OR($t5, $t7)], expr#9=[AND($t6, $t8)], SAL=[$t1], $condition=[$t9])
+EnumerableCalc(expr#0..3=[{inputs}], expr#4=[IS NULL($t3)], SAL=[$t1], $condition=[$t4])
   EnumerableJoin(condition=[true], joinType=[left])
     EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], SAL=[$t5])
       EnumerableTableScan(table=[[scott, EMP]])
@@ -1524,7 +1523,7 @@ select sal from "scott".emp
 (0 rows)
 
 !ok
-EnumerableCalc(expr#0..3=[{inputs}], expr#4=[IS NOT FALSE($t2)], expr#5=[IS NULL($t3)], expr#6=[OR($t4, $t5)], expr#7=[NOT($t2)], expr#8=[OR($t5, $t7)], expr#9=[AND($t6, $t8)], SAL=[$t1], $condition=[$t9])
+EnumerableCalc(expr#0..3=[{inputs}], expr#4=[IS NULL($t3)], SAL=[$t1], $condition=[$t4])
   EnumerableJoin(condition=[true], joinType=[left])
     EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], SAL=[$t5])
       EnumerableTableScan(table=[[scott, EMP]])
@@ -1545,7 +1544,7 @@ select sal from "scott".emp
 (0 rows)
 
 !ok
-EnumerableCalc(expr#0..3=[{inputs}], expr#4=[NOT($t2)], expr#5=[IS NOT NULL($t2)], expr#6=[OR($t4, $t5)], expr#7=[IS NOT TRUE($t6)], expr#8=[IS NULL($t3)], expr#9=[OR($t7, $t8)], SAL=[$t1], $condition=[$t9])
+EnumerableCalc(expr#0..3=[{inputs}], expr#4=[IS NULL($t3)], expr#5=[NOT($t2)], expr#6=[IS NOT NULL($t2)], expr#7=[OR($t5, $t6)], expr#8=[IS NOT TRUE($t7)], expr#9=[OR($t4, $t8)], SAL=[$t1], $condition=[$t9])
   EnumerableJoin(condition=[true], joinType=[left])
     EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], SAL=[$t5])
       EnumerableTableScan(table=[[scott, EMP]])
@@ -1566,7 +1565,7 @@ select sal from "scott".emp
 (0 rows)
 
 !ok
-EnumerableCalc(expr#0..3=[{inputs}], expr#4=[NOT($t2)], expr#5=[IS NOT NULL($t2)], expr#6=[OR($t4, $t5)], expr#7=[IS NOT TRUE($t6)], expr#8=[IS NULL($t3)], expr#9=[OR($t7, $t8)], SAL=[$t1], $condition=[$t9])
+EnumerableCalc(expr#0..3=[{inputs}], expr#4=[IS NULL($t3)], expr#5=[NOT($t2)], expr#6=[IS NOT NULL($t2)], expr#7=[OR($t5, $t6)], expr#8=[IS NOT TRUE($t7)], expr#9=[OR($t4, $t8)], SAL=[$t1], $condition=[$t9])
   EnumerableJoin(condition=[true], joinType=[left])
     EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], SAL=[$t5])
       EnumerableTableScan(table=[[scott, EMP]])
@@ -2048,15 +2047,14 @@ where sal + 100 not in (
 
 !ok
 EnumerableAggregate(group=[{}], C=[COUNT()])
-  EnumerableCalc(expr#0..9=[{inputs}], expr#10=[IS NOT NULL($t7)], expr#11=[<($t5, $t4)], expr#12=[OR($t10, $t11)], expr#13=[IS NOT TRUE($t12)], expr#14=[0], expr#15=[=($t4, $t14)], expr#16=[IS NULL($t2)], expr#17=[OR($t13, $t15, $t16)], proj#0..9=[{exprs}], $condition=[$t17])
+  EnumerableCalc(expr#0..9=[{inputs}], expr#10=[0], expr#11=[=($t4, $t10)], expr#12=[IS NOT TRUE($t11)], expr#13=[IS NOT NULL($t7)], expr#14=[<($t5, $t4)], expr#15=[OR($t13, $t14)], expr#16=[IS NOT TRUE($t15)], expr#17=[IS NOT NULL($t2)], expr#18=[AND($t12, $t16, $t17)], expr#19=[OR($t11, $t18)], proj#0..9=[{exprs}], $condition=[$t19])
     EnumerableJoin(condition=[AND(=($1, $8), =($2, $9))], joinType=[left])
-      EnumerableCalc(expr#0..5=[{inputs}], expr#6=[IS NOT NULL($t2)], expr#7=[0], expr#8=[=($t4, $t7)], expr#9=[OR($t6, $t8)], proj#0..5=[{exprs}], $condition=[$t9])
-        EnumerableJoin(condition=[=($1, $3)], joinType=[left])
-          EnumerableCalc(expr#0..7=[{inputs}], proj#0..1=[{exprs}], SAL=[$t5])
-            EnumerableTableScan(table=[[scott, EMP]])
-          EnumerableAggregate(group=[{1}], c=[COUNT()], ck=[COUNT($0)])
-            EnumerableCalc(expr#0..2=[{inputs}], expr#3=[IS NOT NULL($t1)], proj#0..2=[{exprs}], $condition=[$t3])
-              EnumerableTableScan(table=[[scott, DEPT]])
+      EnumerableJoin(condition=[=($1, $3)], joinType=[left])
+        EnumerableCalc(expr#0..7=[{inputs}], proj#0..1=[{exprs}], SAL=[$t5])
+          EnumerableTableScan(table=[[scott, EMP]])
+        EnumerableAggregate(group=[{1}], c=[COUNT()], ck=[COUNT($0)])
+          EnumerableCalc(expr#0..2=[{inputs}], expr#3=[IS NOT NULL($t1)], proj#0..2=[{exprs}], $condition=[$t3])
+            EnumerableTableScan(table=[[scott, DEPT]])
       EnumerableCalc(expr#0..4=[{inputs}], DEPTNO=[$t2], i=[$t3], DNAME=[$t4], SAL=[$t0])
         EnumerableJoin(condition=[=($1, $2)], joinType=[inner])
           EnumerableCalc(expr#0=[{inputs}], expr#1=[100], expr#2=[+($t0, $t1)], SAL=[$t0], $f1=[$t2])