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

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


The following commit(s) were added to refs/heads/master by this push:
     new 32d4a2b  [SPARK-33861][SQL] Simplify conditional in predicate
32d4a2b is described below

commit 32d4a2b06220861efda1058b26d9a2ed3a1b2c74
Author: Yuming Wang <yumw...@ebay.com>
AuthorDate: Thu Dec 24 08:10:28 2020 +0000

    [SPARK-33861][SQL] Simplify conditional in predicate
    
    ### What changes were proposed in this pull request?
    
    This pr simplify conditional in predicate, after this change we can push 
down the filter to datasource:
    
    Expression | After simplify
    -- | --
    IF(cond, trueVal, false)                   | AND(cond, trueVal)
    IF(cond, trueVal, true)                    | OR(NOT(cond), trueVal)
    IF(cond, false, falseVal)                  | AND(NOT(cond), elseVal)
    IF(cond, true, falseVal)                   | OR(cond, elseVal)
    CASE WHEN cond THEN trueVal ELSE false END | AND(cond, trueVal)
    CASE WHEN cond THEN trueVal END            | AND(cond, trueVal)
    CASE WHEN cond THEN trueVal ELSE null END  | AND(cond, trueVal)
    CASE WHEN cond THEN trueVal ELSE true END  | OR(NOT(cond), trueVal)
    CASE WHEN cond THEN false ELSE elseVal END | AND(NOT(cond), elseVal)
    CASE WHEN cond THEN false END              | false
    CASE WHEN cond THEN true ELSE elseVal END  | OR(cond, elseVal)
    CASE WHEN cond THEN true END               | cond
    
    ### Why are the changes needed?
    
    Improve query performance.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No.
    
    ### How was this patch tested?
    
    Unit test.
    
    Closes #30865 from wangyum/SPARK-33861.
    
    Authored-by: Yuming Wang <yumw...@ebay.com>
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>
---
 .../spark/sql/catalyst/optimizer/Optimizer.scala   |   1 +
 .../SimplifyConditionalsInPredicate.scala          |  82 +++++++
 .../SimplifyConditionalsInPredicateSuite.scala     | 237 +++++++++++++++++++++
 .../approved-plans-modified/q34.sf100/explain.txt  |  12 +-
 .../q34.sf100/simplified.txt                       |   2 +-
 .../approved-plans-modified/q34/explain.txt        |   8 +-
 .../approved-plans-modified/q34/simplified.txt     |   2 +-
 .../approved-plans-modified/q73.sf100/explain.txt  |   8 +-
 .../q73.sf100/simplified.txt                       |   2 +-
 .../approved-plans-modified/q73/explain.txt        |   8 +-
 .../approved-plans-modified/q73/simplified.txt     |   2 +-
 .../approved-plans-v1_4/q34.sf100/explain.txt      |  12 +-
 .../approved-plans-v1_4/q34.sf100/simplified.txt   |   2 +-
 .../approved-plans-v1_4/q34/explain.txt            |   8 +-
 .../approved-plans-v1_4/q34/simplified.txt         |   2 +-
 .../approved-plans-v1_4/q73.sf100/explain.txt      |  12 +-
 .../approved-plans-v1_4/q73.sf100/simplified.txt   |   2 +-
 .../approved-plans-v1_4/q73/explain.txt            |   8 +-
 .../approved-plans-v1_4/q73/simplified.txt         |   2 +-
 .../approved-plans-v2_7/q34.sf100/explain.txt      |  12 +-
 .../approved-plans-v2_7/q34.sf100/simplified.txt   |   2 +-
 .../approved-plans-v2_7/q34/explain.txt            |   8 +-
 .../approved-plans-v2_7/q34/simplified.txt         |   2 +-
 23 files changed, 378 insertions(+), 58 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 61bcf90..7b9b99b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -103,6 +103,7 @@ abstract class Optimizer(catalogManager: CatalogManager)
         RemoveDispensableExpressions,
         SimplifyBinaryComparison,
         ReplaceNullWithFalseInPredicate,
+        SimplifyConditionalsInPredicate,
         PruneFilters,
         SimplifyCasts,
         SimplifyCaseConversionExpressions,
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicate.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicate.scala
new file mode 100644
index 0000000..1ea8508
--- /dev/null
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicate.scala
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{And, CaseWhen, Expression, 
If, Literal, Not, Or}
+import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, 
TrueLiteral}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * A rule that converts conditional expressions to predicate expressions, if 
possible, in the
+ * search condition of the WHERE/HAVING/ON(JOIN) clauses, which contain an 
implicit Boolean operator
+ * "(search condition) = TRUE". After this converting, we can potentially push 
the filter down to
+ * the data source.
+ *
+ * Supported cases are:
+ * - IF(cond, trueVal, false)                   => AND(cond, trueVal)
+ * - IF(cond, trueVal, true)                    => OR(NOT(cond), trueVal)
+ * - IF(cond, false, falseVal)                  => AND(NOT(cond), elseVal)
+ * - IF(cond, true, falseVal)                   => OR(cond, elseVal)
+ * - CASE WHEN cond THEN trueVal ELSE false END => AND(cond, trueVal)
+ * - CASE WHEN cond THEN trueVal END            => AND(cond, trueVal)
+ * - CASE WHEN cond THEN trueVal ELSE null END  => AND(cond, trueVal)
+ * - CASE WHEN cond THEN trueVal ELSE true END  => OR(NOT(cond), trueVal)
+ * - CASE WHEN cond THEN false ELSE elseVal END => AND(NOT(cond), elseVal)
+ * - CASE WHEN cond THEN false END              => false
+ * - CASE WHEN cond THEN true ELSE elseVal END  => OR(cond, elseVal)
+ * - CASE WHEN cond THEN true END               => cond
+ */
+object SimplifyConditionalsInPredicate extends Rule[LogicalPlan] {
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case f @ Filter(cond, _) => f.copy(condition = simplifyConditional(cond))
+    case j @ Join(_, _, _, Some(cond), _) => j.copy(condition = 
Some(simplifyConditional(cond)))
+    case d @ DeleteFromTable(_, Some(cond)) => d.copy(condition = 
Some(simplifyConditional(cond)))
+    case u @ UpdateTable(_, _, Some(cond)) => u.copy(condition = 
Some(simplifyConditional(cond)))
+  }
+
+  private def simplifyConditional(e: Expression): Expression = e match {
+    case And(left, right) => And(simplifyConditional(left), 
simplifyConditional(right))
+    case Or(left, right) => Or(simplifyConditional(left), 
simplifyConditional(right))
+    case If(cond, trueValue, FalseLiteral) => And(cond, trueValue)
+    case If(cond, trueValue, TrueLiteral) => Or(Not(cond), trueValue)
+    case If(cond, FalseLiteral, falseValue) => And(Not(cond), falseValue)
+    case If(cond, TrueLiteral, falseValue) => Or(cond, falseValue)
+    case CaseWhen(Seq((cond, trueValue)),
+        Some(FalseLiteral) | Some(Literal(null, BooleanType)) | None) =>
+      And(cond, trueValue)
+    case CaseWhen(Seq((cond, trueValue)), Some(TrueLiteral)) =>
+      Or(Not(cond), trueValue)
+    case CaseWhen(Seq((_, FalseLiteral)), Some(FalseLiteral) | None) =>
+      FalseLiteral
+    case CaseWhen(Seq((cond, FalseLiteral)), Some(elseValue)) =>
+      And(Not(cond), elseValue)
+    case CaseWhen(Seq((cond, TrueLiteral)), Some(FalseLiteral) | None) =>
+      cond
+    case CaseWhen(Seq((cond, TrueLiteral)), Some(elseValue)) =>
+      Or(cond, elseValue)
+    case e if e.dataType == BooleanType => e
+    case e =>
+      assert(e.dataType != BooleanType,
+      "Expected a Boolean type expression in SimplifyConditionalsInPredicate, 
" +
+        s"but got the type `${e.dataType.catalogString}` in `${e.sql}`.")
+      e
+  }
+}
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicateSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicateSuite.scala
new file mode 100644
index 0000000..1f3c24b
--- /dev/null
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicateSuite.scala
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{And, CaseWhen, Expression, 
If, IsNotNull, Literal, Or}
+import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, 
TrueLiteral}
+import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, 
LocalRelation, LogicalPlan, UpdateTable}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types.{BooleanType, IntegerType}
+
+class SimplifyConditionalsInPredicateSuite extends PlanTest {
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+    val batches =
+      Batch("SimplifyConditionalsInPredicate", FixedPoint(10),
+        NullPropagation,
+        ConstantFolding,
+        BooleanSimplification,
+        SimplifyConditionals,
+        SimplifyConditionalsInPredicate) :: Nil
+  }
+
+  private val testRelation =
+    LocalRelation('i.int, 'b.boolean, 'a.array(IntegerType), 
'm.map(IntegerType, IntegerType))
+  private val anotherTestRelation = LocalRelation('d.int)
+
+  test("IF(cond, trueVal, false) => AND(cond, trueVal)") {
+    val originalCond = If(
+      UnresolvedAttribute("i") > Literal(10),
+      UnresolvedAttribute("b"),
+      FalseLiteral)
+    val expectedCond = And(
+      UnresolvedAttribute("i") > Literal(10),
+      UnresolvedAttribute("b"))
+    testFilter(originalCond, expectedCond = expectedCond)
+    testJoin(originalCond, expectedCond = expectedCond)
+    testDelete(originalCond, expectedCond = expectedCond)
+    testUpdate(originalCond, expectedCond = expectedCond)
+    testProjection(originalCond, expectedExpr = originalCond)
+  }
+
+  test("IF(cond, trueVal, true) => OR(NOT(cond), trueVal)") {
+    val originalCond = If(
+      UnresolvedAttribute("i") > Literal(10),
+      UnresolvedAttribute("b"),
+      TrueLiteral)
+    val expectedCond = Or(
+      UnresolvedAttribute("i") <= Literal(10),
+      UnresolvedAttribute("b"))
+    testFilter(originalCond, expectedCond = expectedCond)
+    testJoin(originalCond, expectedCond = expectedCond)
+    testDelete(originalCond, expectedCond = expectedCond)
+    testUpdate(originalCond, expectedCond = expectedCond)
+    testProjection(originalCond, expectedExpr = originalCond)
+  }
+
+  test("IF(cond, false, falseVal) => AND(NOT(cond), elseVal)") {
+    val originalCond = If(
+      UnresolvedAttribute("i") > Literal(10),
+      FalseLiteral,
+      UnresolvedAttribute("b"))
+    val expectedCond = And(
+      UnresolvedAttribute("i") <= Literal(10),
+      UnresolvedAttribute("b"))
+    testFilter(originalCond, expectedCond = expectedCond)
+    testJoin(originalCond, expectedCond = expectedCond)
+    testDelete(originalCond, expectedCond = expectedCond)
+    testUpdate(originalCond, expectedCond = expectedCond)
+    testProjection(originalCond, expectedExpr = originalCond)
+  }
+
+  test("IF(cond, true, falseVal) => OR(cond, elseVal)") {
+    val originalCond = If(
+      UnresolvedAttribute("i") > Literal(10),
+      TrueLiteral,
+      UnresolvedAttribute("b"))
+    val expectedCond = Or(
+      UnresolvedAttribute("i") > Literal(10),
+      UnresolvedAttribute("b"))
+    testFilter(originalCond, expectedCond = expectedCond)
+    testJoin(originalCond, expectedCond = expectedCond)
+    testDelete(originalCond, expectedCond = expectedCond)
+    testUpdate(originalCond, expectedCond = expectedCond)
+    testProjection(originalCond, expectedExpr = originalCond)
+  }
+
+  test("CASE WHEN cond THEN trueVal ELSE false END => AND(cond, trueVal)") {
+    Seq(Some(FalseLiteral), None, Some(Literal(null, BooleanType))).foreach { 
elseExp =>
+      val originalCond = CaseWhen(
+        Seq((UnresolvedAttribute("i") > Literal(10), 
UnresolvedAttribute("b"))),
+        elseExp)
+      val expectedCond = And(
+        UnresolvedAttribute("i") > Literal(10),
+        UnresolvedAttribute("b"))
+      testFilter(originalCond, expectedCond = expectedCond)
+      testJoin(originalCond, expectedCond = expectedCond)
+      testDelete(originalCond, expectedCond = expectedCond)
+      testUpdate(originalCond, expectedCond = expectedCond)
+      testProjection(originalCond, expectedExpr = originalCond)
+    }
+  }
+
+  test("CASE WHEN cond THEN trueVal ELSE true END => OR(NOT(cond), trueVal)") {
+    val originalCond = CaseWhen(
+      Seq((UnresolvedAttribute("i") > Literal(10), UnresolvedAttribute("b"))),
+      TrueLiteral)
+    val expectedCond = Or(
+      UnresolvedAttribute("i") <= Literal(10),
+      UnresolvedAttribute("b"))
+    testFilter(originalCond, expectedCond = expectedCond)
+    testJoin(originalCond, expectedCond = expectedCond)
+    testDelete(originalCond, expectedCond = expectedCond)
+    testUpdate(originalCond, expectedCond = expectedCond)
+    testProjection(originalCond, expectedExpr = originalCond)
+  }
+
+  test("CASE WHEN cond THEN false ELSE elseVal END => AND(NOT(cond), 
elseVal)") {
+    val originalCond = CaseWhen(
+      Seq((UnresolvedAttribute("i") > Literal(10), FalseLiteral)),
+      UnresolvedAttribute("b"))
+    val expectedCond = And(
+      UnresolvedAttribute("i") <= Literal(10),
+      UnresolvedAttribute("b"))
+    testFilter(originalCond, expectedCond = expectedCond)
+    testJoin(originalCond, expectedCond = expectedCond)
+    testDelete(originalCond, expectedCond = expectedCond)
+    testUpdate(originalCond, expectedCond = expectedCond)
+    testProjection(originalCond, expectedExpr = originalCond)
+  }
+
+  test("CASE WHEN cond THEN false END => false") {
+    val originalCond = CaseWhen(
+      Seq((UnresolvedAttribute("i") > Literal(10), FalseLiteral)))
+    testFilter(originalCond, expectedCond = FalseLiteral)
+    testJoin(originalCond, expectedCond = FalseLiteral)
+    testDelete(originalCond, expectedCond = FalseLiteral)
+    testUpdate(originalCond, expectedCond = FalseLiteral)
+    testProjection(originalCond, expectedExpr = originalCond)
+  }
+
+  test("CASE WHEN cond THEN true ELSE elseVal END  => OR(cond, elseVal)") {
+    val originalCond = CaseWhen(
+      Seq((UnresolvedAttribute("i") > Literal(10), TrueLiteral)),
+      UnresolvedAttribute("b"))
+    val expectedCond = Or(
+      UnresolvedAttribute("i") > Literal(10),
+      UnresolvedAttribute("b"))
+    testFilter(originalCond, expectedCond = expectedCond)
+    testJoin(originalCond, expectedCond = expectedCond)
+    testDelete(originalCond, expectedCond = expectedCond)
+    testUpdate(originalCond, expectedCond = expectedCond)
+    testProjection(originalCond, expectedExpr = originalCond)
+  }
+
+  test("CASE WHEN cond THEN true END => cond") {
+    val originalCond = CaseWhen(
+      Seq((UnresolvedAttribute("i") > Literal(10), TrueLiteral)))
+    val expectedCond = UnresolvedAttribute("i") > Literal(10)
+    testFilter(originalCond, expectedCond = expectedCond)
+    testJoin(originalCond, expectedCond = expectedCond)
+    testDelete(originalCond, expectedCond = expectedCond)
+    testUpdate(originalCond, expectedCond = expectedCond)
+    testProjection(originalCond, expectedExpr = originalCond)
+  }
+
+  test("Simplify conditional in conditions of CaseWhen inside another 
CaseWhen") {
+    val nestedCaseWhen = CaseWhen(
+      Seq((UnresolvedAttribute("i") > Literal(10)) -> 
UnresolvedAttribute("b")),
+      FalseLiteral)
+    val originalCond = CaseWhen(Seq(IsNotNull(nestedCaseWhen) -> FalseLiteral))
+    val expectedCond = FalseLiteral
+
+    testFilter(originalCond, expectedCond = expectedCond)
+    testJoin(originalCond, expectedCond = expectedCond)
+    testDelete(originalCond, expectedCond = expectedCond)
+    testUpdate(originalCond, expectedCond = expectedCond)
+    testProjection(originalCond, expectedExpr = originalCond)
+  }
+
+  test("Not expected type - SimplifyConditionalsInPredicate") {
+    val e = intercept[AnalysisException] {
+      testFilter(originalCond = Literal(null, IntegerType), expectedCond = 
FalseLiteral)
+    }.getMessage
+    assert(e.contains("'CAST(NULL AS INT)' of type int is not a boolean"))
+  }
+
+  private def testFilter(originalCond: Expression, expectedCond: Expression): 
Unit = {
+    test((rel, exp) => rel.where(exp), originalCond, expectedCond)
+  }
+
+  private def testJoin(originalCond: Expression, expectedCond: Expression): 
Unit = {
+    test((rel, exp) => rel.join(anotherTestRelation, Inner, Some(exp)), 
originalCond, expectedCond)
+  }
+
+  private def testProjection(originalExpr: Expression, expectedExpr: 
Expression): Unit = {
+    test((rel, exp) => rel.select(exp), originalExpr, expectedExpr)
+  }
+
+  private def testDelete(originalCond: Expression, expectedCond: Expression): 
Unit = {
+    test((rel, expr) => DeleteFromTable(rel, Some(expr)), originalCond, 
expectedCond)
+  }
+
+  private def testUpdate(originalCond: Expression, expectedCond: Expression): 
Unit = {
+    test((rel, expr) => UpdateTable(rel, Seq.empty, Some(expr)), originalCond, 
expectedCond)
+  }
+
+  private def test(
+      func: (LogicalPlan, Expression) => LogicalPlan,
+      originalExpr: Expression,
+      expectedExpr: Expression): Unit = {
+
+    val originalPlan = func(testRelation, originalExpr).analyze
+    val optimizedPlan = Optimize.execute(originalPlan)
+    val expectedPlan = func(testRelation, expectedExpr).analyze
+    comparePlans(optimizedPlan, expectedPlan)
+  }
+}
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt
index ac1fca4..5478061 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt
@@ -120,7 +120,7 @@ Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, 
ss_ticket_number#5,
 Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/household_demographics]
-PushedFilters: [IsNotNull(hd_vehicle_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), 
GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)]
+PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), 
GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), 
IsNotNull(hd_demo_sk)]
 ReadSchema: 
struct<hd_demo_sk:int,hd_buy_potential:string,hd_dep_count:int,hd_vehicle_count:int>
 
 (19) ColumnarToRow [codegen id : 3]
@@ -128,7 +128,7 @@ Input [4]: [hd_demo_sk#13, hd_buy_potential#14, 
hd_dep_count#15, hd_vehicle_coun
 
 (20) Filter [codegen id : 3]
 Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
-Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = 
>10000) OR (hd_buy_potential#14 = Unknown))) AND (hd_vehicle_count#16 > 0)) AND 
(CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / 
cast(hd_vehicle_count#16 as double)) ELSE null END > 1.2)) AND 
isnotnull(hd_demo_sk#13))
+Condition : (((((isnotnull(hd_vehicle_count#16) AND 
isnotnull(hd_dep_count#15)) AND ((hd_buy_potential#14 = >10000) OR 
(hd_buy_potential#14 = Unknown))) AND (hd_vehicle_count#16 > 0)) AND 
((cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) > 
1.2)) AND isnotnull(hd_demo_sk#13))
 
 (21) Project [codegen id : 3]
 Output [1]: [hd_demo_sk#13]
@@ -156,7 +156,7 @@ Results [3]: [ss_ticket_number#5, ss_customer_sk#2, 
count#19]
 
 (26) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
-Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, 
[id=#20]
+Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), 
ENSURE_REQUIREMENTS, [id=#20]
 
 (27) HashAggregate [codegen id : 5]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
@@ -171,7 +171,7 @@ Condition : ((cnt#22 >= 15) AND (cnt#22 <= 20))
 
 (29) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22]
-Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#23]
+Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23]
 
 (30) Sort [codegen id : 6]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22]
@@ -193,7 +193,7 @@ Condition : isnotnull(c_customer_sk#24)
 
 (34) Exchange
 Input [5]: [c_customer_sk#24, c_salutation#25, c_first_name#26, 
c_last_name#27, c_preferred_cust_flag#28]
-Arguments: hashpartitioning(c_customer_sk#24, 5), true, [id=#29]
+Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [id=#29]
 
 (35) Sort [codegen id : 8]
 Input [5]: [c_customer_sk#24, c_salutation#25, c_first_name#26, 
c_last_name#27, c_preferred_cust_flag#28]
@@ -210,7 +210,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, 
c_customer_sk#24, c_sa
 
 (38) Exchange
 Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, 
c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22]
-Arguments: rangepartitioning(c_last_name#27 ASC NULLS FIRST, c_first_name#26 
ASC NULLS FIRST, c_salutation#25 ASC NULLS FIRST, c_preferred_cust_flag#28 DESC 
NULLS LAST, 5), true, [id=#30]
+Arguments: rangepartitioning(c_last_name#27 ASC NULLS FIRST, c_first_name#26 
ASC NULLS FIRST, c_salutation#25 ASC NULLS FIRST, c_preferred_cust_flag#28 DESC 
NULLS LAST, 5), ENSURE_REQUIREMENTS, [id=#30]
 
 (39) Sort [codegen id : 10]
 Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, 
c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/simplified.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/simplified.txt
index d9b416d..c9945cd 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/simplified.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/simplified.txt
@@ -47,7 +47,7 @@ WholeStageCodegen (10)
                                             BroadcastExchange #6
                                               WholeStageCodegen (3)
                                                 Project [hd_demo_sk]
-                                                  Filter 
[hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+                                                  Filter 
[hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk]
                                                     ColumnarToRow
                                                       InputAdapter
                                                         Scan parquet 
default.household_demographics 
[hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt
index 898d374..74bbb52 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt
@@ -117,7 +117,7 @@ Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, 
ss_ticket_number#5,
 Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/household_demographics]
-PushedFilters: [IsNotNull(hd_vehicle_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), 
GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)]
+PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), 
GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), 
IsNotNull(hd_demo_sk)]
 ReadSchema: 
struct<hd_demo_sk:int,hd_buy_potential:string,hd_dep_count:int,hd_vehicle_count:int>
 
 (19) ColumnarToRow [codegen id : 3]
@@ -125,7 +125,7 @@ Input [4]: [hd_demo_sk#13, hd_buy_potential#14, 
hd_dep_count#15, hd_vehicle_coun
 
 (20) Filter [codegen id : 3]
 Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
-Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = 
>10000) OR (hd_buy_potential#14 = Unknown))) AND (hd_vehicle_count#16 > 0)) AND 
(CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / 
cast(hd_vehicle_count#16 as double)) ELSE null END > 1.2)) AND 
isnotnull(hd_demo_sk#13))
+Condition : (((((isnotnull(hd_vehicle_count#16) AND 
isnotnull(hd_dep_count#15)) AND ((hd_buy_potential#14 = >10000) OR 
(hd_buy_potential#14 = Unknown))) AND (hd_vehicle_count#16 > 0)) AND 
((cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) > 
1.2)) AND isnotnull(hd_demo_sk#13))
 
 (21) Project [codegen id : 3]
 Output [1]: [hd_demo_sk#13]
@@ -153,7 +153,7 @@ Results [3]: [ss_ticket_number#5, ss_customer_sk#2, 
count#19]
 
 (26) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
-Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, 
[id=#20]
+Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), 
ENSURE_REQUIREMENTS, [id=#20]
 
 (27) HashAggregate [codegen id : 6]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
@@ -195,7 +195,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, 
c_customer_sk#23, c_sa
 
 (35) Exchange
 Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, 
c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22]
-Arguments: rangepartitioning(c_last_name#26 ASC NULLS FIRST, c_first_name#25 
ASC NULLS FIRST, c_salutation#24 ASC NULLS FIRST, c_preferred_cust_flag#27 DESC 
NULLS LAST, 5), true, [id=#29]
+Arguments: rangepartitioning(c_last_name#26 ASC NULLS FIRST, c_first_name#25 
ASC NULLS FIRST, c_salutation#24 ASC NULLS FIRST, c_preferred_cust_flag#27 DESC 
NULLS LAST, 5), ENSURE_REQUIREMENTS, [id=#29]
 
 (36) Sort [codegen id : 7]
 Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, 
c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/simplified.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/simplified.txt
index 5af07f1..4484587 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/simplified.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/simplified.txt
@@ -41,7 +41,7 @@ WholeStageCodegen (7)
                                 BroadcastExchange #5
                                   WholeStageCodegen (3)
                                     Project [hd_demo_sk]
-                                      Filter 
[hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+                                      Filter 
[hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk]
                                         ColumnarToRow
                                           InputAdapter
                                             Scan parquet 
default.household_demographics 
[hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt
index 25da173..51b480e 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt
@@ -117,7 +117,7 @@ Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, 
ss_ticket_number#5,
 Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/household_demographics]
-PushedFilters: [IsNotNull(hd_vehicle_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), 
GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)]
+PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), 
GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), 
IsNotNull(hd_demo_sk)]
 ReadSchema: 
struct<hd_demo_sk:int,hd_buy_potential:string,hd_dep_count:int,hd_vehicle_count:int>
 
 (19) ColumnarToRow [codegen id : 3]
@@ -125,7 +125,7 @@ Input [4]: [hd_demo_sk#13, hd_buy_potential#14, 
hd_dep_count#15, hd_vehicle_coun
 
 (20) Filter [codegen id : 3]
 Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
-Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = 
>10000) OR (hd_buy_potential#14 = Unknown))) AND (hd_vehicle_count#16 > 0)) AND 
(CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / 
cast(hd_vehicle_count#16 as double)) ELSE null END > 1.0)) AND 
isnotnull(hd_demo_sk#13))
+Condition : (((((isnotnull(hd_vehicle_count#16) AND 
isnotnull(hd_dep_count#15)) AND ((hd_buy_potential#14 = >10000) OR 
(hd_buy_potential#14 = Unknown))) AND (hd_vehicle_count#16 > 0)) AND 
((cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) > 
1.0)) AND isnotnull(hd_demo_sk#13))
 
 (21) Project [codegen id : 3]
 Output [1]: [hd_demo_sk#13]
@@ -153,7 +153,7 @@ Results [3]: [ss_ticket_number#5, ss_customer_sk#2, 
count#19]
 
 (26) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
-Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, 
[id=#20]
+Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), 
ENSURE_REQUIREMENTS, [id=#20]
 
 (27) HashAggregate [codegen id : 5]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
@@ -195,7 +195,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, 
c_customer_sk#24, c_sa
 
 (35) Exchange
 Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, 
c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22]
-Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), true, [id=#29]
+Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, 
[id=#29]
 
 (36) Sort [codegen id : 7]
 Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, 
c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/simplified.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/simplified.txt
index 7496388..8695f9d 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/simplified.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/simplified.txt
@@ -44,7 +44,7 @@ WholeStageCodegen (7)
                                       BroadcastExchange #6
                                         WholeStageCodegen (3)
                                           Project [hd_demo_sk]
-                                            Filter 
[hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+                                            Filter 
[hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk]
                                               ColumnarToRow
                                                 InputAdapter
                                                   Scan parquet 
default.household_demographics 
[hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt
index e420b65..56ad4f4 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt
@@ -117,7 +117,7 @@ Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, 
ss_ticket_number#5,
 Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/household_demographics]
-PushedFilters: [IsNotNull(hd_vehicle_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), 
GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)]
+PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(`hd_dep_count`), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), 
GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), 
IsNotNull(hd_demo_sk)]
 ReadSchema: 
struct<hd_demo_sk:int,hd_buy_potential:string,hd_dep_count:int,hd_vehicle_count:int>
 
 (19) ColumnarToRow [codegen id : 3]
@@ -125,7 +125,7 @@ Input [4]: [hd_demo_sk#13, hd_buy_potential#14, 
hd_dep_count#15, hd_vehicle_coun
 
 (20) Filter [codegen id : 3]
 Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
-Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = 
>10000) OR (hd_buy_potential#14 = Unknown))) AND (hd_vehicle_count#16 > 0)) AND 
(CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / 
cast(hd_vehicle_count#16 as double)) ELSE null END > 1.0)) AND 
isnotnull(hd_demo_sk#13))
+Condition : (((((isnotnull(hd_vehicle_count#16) AND 
isnotnull(hd_dep_count#15)) AND ((hd_buy_potential#14 = >10000) OR 
(hd_buy_potential#14 = Unknown))) AND (hd_vehicle_count#16 > 0)) AND 
((cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) > 
1.0)) AND isnotnull(hd_demo_sk#13))
 
 (21) Project [codegen id : 3]
 Output [1]: [hd_demo_sk#13]
@@ -153,7 +153,7 @@ Results [3]: [ss_ticket_number#5, ss_customer_sk#2, 
count#19]
 
 (26) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
-Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, 
[id=#20]
+Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), 
ENSURE_REQUIREMENTS, [id=#20]
 
 (27) HashAggregate [codegen id : 6]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
@@ -195,7 +195,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, 
c_customer_sk#23, c_sa
 
 (35) Exchange
 Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, 
c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22]
-Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), true, [id=#29]
+Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, 
[id=#29]
 
 (36) Sort [codegen id : 7]
 Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, 
c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/simplified.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/simplified.txt
index 46b7241..5e49f6c 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/simplified.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/simplified.txt
@@ -41,7 +41,7 @@ WholeStageCodegen (7)
                                 BroadcastExchange #5
                                   WholeStageCodegen (3)
                                     Project [hd_demo_sk]
-                                      Filter 
[hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+                                      Filter 
[hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk]
                                         ColumnarToRow
                                           InputAdapter
                                             Scan parquet 
default.household_demographics 
[hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt
index 17bb0e7..6fa9bb8 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt
@@ -120,7 +120,7 @@ Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, 
ss_ticket_number#5,
 Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/household_demographics]
-PushedFilters: [IsNotNull(hd_vehicle_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), 
GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)]
+PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), 
GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), 
IsNotNull(hd_demo_sk)]
 ReadSchema: 
struct<hd_demo_sk:int,hd_buy_potential:string,hd_dep_count:int,hd_vehicle_count:int>
 
 (19) ColumnarToRow [codegen id : 3]
@@ -128,7 +128,7 @@ Input [4]: [hd_demo_sk#13, hd_buy_potential#14, 
hd_dep_count#15, hd_vehicle_coun
 
 (20) Filter [codegen id : 3]
 Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
-Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = 
>10000) OR (hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND 
(CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / 
cast(hd_vehicle_count#16 as double)) ELSE null END > 1.2)) AND 
isnotnull(hd_demo_sk#13))
+Condition : (((((isnotnull(hd_vehicle_count#16) AND 
isnotnull(hd_dep_count#15)) AND ((hd_buy_potential#14 = >10000) OR 
(hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND 
((cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) > 
1.2)) AND isnotnull(hd_demo_sk#13))
 
 (21) Project [codegen id : 3]
 Output [1]: [hd_demo_sk#13]
@@ -156,7 +156,7 @@ Results [3]: [ss_ticket_number#5, ss_customer_sk#2, 
count#19]
 
 (26) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
-Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, 
[id=#20]
+Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), 
ENSURE_REQUIREMENTS, [id=#20]
 
 (27) HashAggregate [codegen id : 5]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
@@ -171,7 +171,7 @@ Condition : ((cnt#22 >= 15) AND (cnt#22 <= 20))
 
 (29) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22]
-Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#23]
+Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23]
 
 (30) Sort [codegen id : 6]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22]
@@ -193,7 +193,7 @@ Condition : isnotnull(c_customer_sk#24)
 
 (34) Exchange
 Input [5]: [c_customer_sk#24, c_salutation#25, c_first_name#26, 
c_last_name#27, c_preferred_cust_flag#28]
-Arguments: hashpartitioning(c_customer_sk#24, 5), true, [id=#29]
+Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [id=#29]
 
 (35) Sort [codegen id : 8]
 Input [5]: [c_customer_sk#24, c_salutation#25, c_first_name#26, 
c_last_name#27, c_preferred_cust_flag#28]
@@ -210,7 +210,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, 
c_customer_sk#24, c_sa
 
 (38) Exchange
 Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, 
c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22]
-Arguments: rangepartitioning(c_last_name#27 ASC NULLS FIRST, c_first_name#26 
ASC NULLS FIRST, c_salutation#25 ASC NULLS FIRST, c_preferred_cust_flag#28 DESC 
NULLS LAST, 5), true, [id=#30]
+Arguments: rangepartitioning(c_last_name#27 ASC NULLS FIRST, c_first_name#26 
ASC NULLS FIRST, c_salutation#25 ASC NULLS FIRST, c_preferred_cust_flag#28 DESC 
NULLS LAST, 5), ENSURE_REQUIREMENTS, [id=#30]
 
 (39) Sort [codegen id : 10]
 Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, 
c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/simplified.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/simplified.txt
index d9b416d..c9945cd 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/simplified.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/simplified.txt
@@ -47,7 +47,7 @@ WholeStageCodegen (10)
                                             BroadcastExchange #6
                                               WholeStageCodegen (3)
                                                 Project [hd_demo_sk]
-                                                  Filter 
[hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+                                                  Filter 
[hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk]
                                                     ColumnarToRow
                                                       InputAdapter
                                                         Scan parquet 
default.household_demographics 
[hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt
index 18f465c..1aea774 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt
@@ -117,7 +117,7 @@ Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, 
ss_ticket_number#5,
 Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/household_demographics]
-PushedFilters: [IsNotNull(hd_vehicle_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), 
GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)]
+PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), 
GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), 
IsNotNull(hd_demo_sk)]
 ReadSchema: 
struct<hd_demo_sk:int,hd_buy_potential:string,hd_dep_count:int,hd_vehicle_count:int>
 
 (19) ColumnarToRow [codegen id : 3]
@@ -125,7 +125,7 @@ Input [4]: [hd_demo_sk#13, hd_buy_potential#14, 
hd_dep_count#15, hd_vehicle_coun
 
 (20) Filter [codegen id : 3]
 Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
-Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = 
>10000) OR (hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND 
(CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / 
cast(hd_vehicle_count#16 as double)) ELSE null END > 1.2)) AND 
isnotnull(hd_demo_sk#13))
+Condition : (((((isnotnull(hd_vehicle_count#16) AND 
isnotnull(hd_dep_count#15)) AND ((hd_buy_potential#14 = >10000) OR 
(hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND 
((cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) > 
1.2)) AND isnotnull(hd_demo_sk#13))
 
 (21) Project [codegen id : 3]
 Output [1]: [hd_demo_sk#13]
@@ -153,7 +153,7 @@ Results [3]: [ss_ticket_number#5, ss_customer_sk#2, 
count#19]
 
 (26) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
-Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, 
[id=#20]
+Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), 
ENSURE_REQUIREMENTS, [id=#20]
 
 (27) HashAggregate [codegen id : 6]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
@@ -195,7 +195,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, 
c_customer_sk#23, c_sa
 
 (35) Exchange
 Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, 
c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22]
-Arguments: rangepartitioning(c_last_name#26 ASC NULLS FIRST, c_first_name#25 
ASC NULLS FIRST, c_salutation#24 ASC NULLS FIRST, c_preferred_cust_flag#27 DESC 
NULLS LAST, 5), true, [id=#29]
+Arguments: rangepartitioning(c_last_name#26 ASC NULLS FIRST, c_first_name#25 
ASC NULLS FIRST, c_salutation#24 ASC NULLS FIRST, c_preferred_cust_flag#27 DESC 
NULLS LAST, 5), ENSURE_REQUIREMENTS, [id=#29]
 
 (36) Sort [codegen id : 7]
 Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, 
c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt
index 5af07f1..4484587 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt
@@ -41,7 +41,7 @@ WholeStageCodegen (7)
                                 BroadcastExchange #5
                                   WholeStageCodegen (3)
                                     Project [hd_demo_sk]
-                                      Filter 
[hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+                                      Filter 
[hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk]
                                         ColumnarToRow
                                           InputAdapter
                                             Scan parquet 
default.household_demographics 
[hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt
index 4af604c..f88f1f4 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt
@@ -120,7 +120,7 @@ Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, 
ss_ticket_number#5,
 Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/household_demographics]
-PushedFilters: [IsNotNull(hd_vehicle_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), 
GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)]
+PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), 
GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), 
IsNotNull(hd_demo_sk)]
 ReadSchema: 
struct<hd_demo_sk:int,hd_buy_potential:string,hd_dep_count:int,hd_vehicle_count:int>
 
 (19) ColumnarToRow [codegen id : 3]
@@ -128,7 +128,7 @@ Input [4]: [hd_demo_sk#13, hd_buy_potential#14, 
hd_dep_count#15, hd_vehicle_coun
 
 (20) Filter [codegen id : 3]
 Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
-Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = 
>10000) OR (hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND 
(CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / 
cast(hd_vehicle_count#16 as double)) ELSE null END > 1.0)) AND 
isnotnull(hd_demo_sk#13))
+Condition : (((((isnotnull(hd_vehicle_count#16) AND 
isnotnull(hd_dep_count#15)) AND ((hd_buy_potential#14 = >10000) OR 
(hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND 
((cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) > 
1.0)) AND isnotnull(hd_demo_sk#13))
 
 (21) Project [codegen id : 3]
 Output [1]: [hd_demo_sk#13]
@@ -156,7 +156,7 @@ Results [3]: [ss_ticket_number#5, ss_customer_sk#2, 
count#19]
 
 (26) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
-Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, 
[id=#20]
+Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), 
ENSURE_REQUIREMENTS, [id=#20]
 
 (27) HashAggregate [codegen id : 5]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
@@ -171,7 +171,7 @@ Condition : ((cnt#22 >= 1) AND (cnt#22 <= 5))
 
 (29) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22]
-Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#23]
+Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23]
 
 (30) Sort [codegen id : 6]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22]
@@ -193,7 +193,7 @@ Condition : isnotnull(c_customer_sk#24)
 
 (34) Exchange
 Input [5]: [c_customer_sk#24, c_salutation#25, c_first_name#26, 
c_last_name#27, c_preferred_cust_flag#28]
-Arguments: hashpartitioning(c_customer_sk#24, 5), true, [id=#29]
+Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [id=#29]
 
 (35) Sort [codegen id : 8]
 Input [5]: [c_customer_sk#24, c_salutation#25, c_first_name#26, 
c_last_name#27, c_preferred_cust_flag#28]
@@ -210,7 +210,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, 
c_customer_sk#24, c_sa
 
 (38) Exchange
 Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, 
c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22]
-Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), true, [id=#30]
+Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, 
[id=#30]
 
 (39) Sort [codegen id : 10]
 Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, 
c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/simplified.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/simplified.txt
index af8527f..9de2f2a 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/simplified.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/simplified.txt
@@ -47,7 +47,7 @@ WholeStageCodegen (10)
                                             BroadcastExchange #6
                                               WholeStageCodegen (3)
                                                 Project [hd_demo_sk]
-                                                  Filter 
[hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+                                                  Filter 
[hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk]
                                                     ColumnarToRow
                                                       InputAdapter
                                                         Scan parquet 
default.household_demographics 
[hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt
index f4565c3..43c73f3 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt
@@ -117,7 +117,7 @@ Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, 
ss_ticket_number#5,
 Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/household_demographics]
-PushedFilters: [IsNotNull(hd_vehicle_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), 
GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)]
+PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), 
GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), 
IsNotNull(hd_demo_sk)]
 ReadSchema: 
struct<hd_demo_sk:int,hd_buy_potential:string,hd_dep_count:int,hd_vehicle_count:int>
 
 (19) ColumnarToRow [codegen id : 3]
@@ -125,7 +125,7 @@ Input [4]: [hd_demo_sk#13, hd_buy_potential#14, 
hd_dep_count#15, hd_vehicle_coun
 
 (20) Filter [codegen id : 3]
 Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
-Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = 
>10000) OR (hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND 
(CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / 
cast(hd_vehicle_count#16 as double)) ELSE null END > 1.0)) AND 
isnotnull(hd_demo_sk#13))
+Condition : (((((isnotnull(hd_vehicle_count#16) AND 
isnotnull(hd_dep_count#15)) AND ((hd_buy_potential#14 = >10000) OR 
(hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND 
((cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) > 
1.0)) AND isnotnull(hd_demo_sk#13))
 
 (21) Project [codegen id : 3]
 Output [1]: [hd_demo_sk#13]
@@ -153,7 +153,7 @@ Results [3]: [ss_ticket_number#5, ss_customer_sk#2, 
count#19]
 
 (26) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
-Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, 
[id=#20]
+Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), 
ENSURE_REQUIREMENTS, [id=#20]
 
 (27) HashAggregate [codegen id : 6]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
@@ -195,7 +195,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, 
c_customer_sk#23, c_sa
 
 (35) Exchange
 Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, 
c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22]
-Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), true, [id=#29]
+Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, 
[id=#29]
 
 (36) Sort [codegen id : 7]
 Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, 
c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt
index 46b7241..5e49f6c 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt
@@ -41,7 +41,7 @@ WholeStageCodegen (7)
                                 BroadcastExchange #5
                                   WholeStageCodegen (3)
                                     Project [hd_demo_sk]
-                                      Filter 
[hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+                                      Filter 
[hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk]
                                         ColumnarToRow
                                           InputAdapter
                                             Scan parquet 
default.household_demographics 
[hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt
index c7b8685..5d8f0d0 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt
@@ -120,7 +120,7 @@ Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, 
ss_ticket_number#5,
 Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/household_demographics]
-PushedFilters: [IsNotNull(hd_vehicle_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), 
GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)]
+PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), 
GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), 
IsNotNull(hd_demo_sk)]
 ReadSchema: 
struct<hd_demo_sk:int,hd_buy_potential:string,hd_dep_count:int,hd_vehicle_count:int>
 
 (19) ColumnarToRow [codegen id : 3]
@@ -128,7 +128,7 @@ Input [4]: [hd_demo_sk#13, hd_buy_potential#14, 
hd_dep_count#15, hd_vehicle_coun
 
 (20) Filter [codegen id : 3]
 Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
-Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = 
>10000) OR (hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND 
(CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / 
cast(hd_vehicle_count#16 as double)) ELSE null END > 1.2)) AND 
isnotnull(hd_demo_sk#13))
+Condition : (((((isnotnull(hd_vehicle_count#16) AND 
isnotnull(hd_dep_count#15)) AND ((hd_buy_potential#14 = >10000) OR 
(hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND 
((cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) > 
1.2)) AND isnotnull(hd_demo_sk#13))
 
 (21) Project [codegen id : 3]
 Output [1]: [hd_demo_sk#13]
@@ -156,7 +156,7 @@ Results [3]: [ss_ticket_number#5, ss_customer_sk#2, 
count#19]
 
 (26) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
-Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, 
[id=#20]
+Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), 
ENSURE_REQUIREMENTS, [id=#20]
 
 (27) HashAggregate [codegen id : 5]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
@@ -171,7 +171,7 @@ Condition : ((cnt#22 >= 15) AND (cnt#22 <= 20))
 
 (29) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22]
-Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#23]
+Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23]
 
 (30) Sort [codegen id : 6]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22]
@@ -193,7 +193,7 @@ Condition : isnotnull(c_customer_sk#24)
 
 (34) Exchange
 Input [5]: [c_customer_sk#24, c_salutation#25, c_first_name#26, 
c_last_name#27, c_preferred_cust_flag#28]
-Arguments: hashpartitioning(c_customer_sk#24, 5), true, [id=#29]
+Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [id=#29]
 
 (35) Sort [codegen id : 8]
 Input [5]: [c_customer_sk#24, c_salutation#25, c_first_name#26, 
c_last_name#27, c_preferred_cust_flag#28]
@@ -210,7 +210,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, 
c_customer_sk#24, c_sa
 
 (38) Exchange
 Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, 
c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22]
-Arguments: rangepartitioning(c_last_name#27 ASC NULLS FIRST, c_first_name#26 
ASC NULLS FIRST, c_salutation#25 ASC NULLS FIRST, c_preferred_cust_flag#28 DESC 
NULLS LAST, ss_ticket_number#5 ASC NULLS FIRST, 5), true, [id=#30]
+Arguments: rangepartitioning(c_last_name#27 ASC NULLS FIRST, c_first_name#26 
ASC NULLS FIRST, c_salutation#25 ASC NULLS FIRST, c_preferred_cust_flag#28 DESC 
NULLS LAST, ss_ticket_number#5 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, 
[id=#30]
 
 (39) Sort [codegen id : 10]
 Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, 
c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/simplified.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/simplified.txt
index 451659e..244478f 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/simplified.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/simplified.txt
@@ -47,7 +47,7 @@ WholeStageCodegen (10)
                                             BroadcastExchange #6
                                               WholeStageCodegen (3)
                                                 Project [hd_demo_sk]
-                                                  Filter 
[hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+                                                  Filter 
[hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk]
                                                     ColumnarToRow
                                                       InputAdapter
                                                         Scan parquet 
default.household_demographics 
[hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt
index 01b5f46..e588993 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt
@@ -117,7 +117,7 @@ Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, 
ss_ticket_number#5,
 Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/household_demographics]
-PushedFilters: [IsNotNull(hd_vehicle_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), 
GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)]
+PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), 
Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), 
GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), 
IsNotNull(hd_demo_sk)]
 ReadSchema: 
struct<hd_demo_sk:int,hd_buy_potential:string,hd_dep_count:int,hd_vehicle_count:int>
 
 (19) ColumnarToRow [codegen id : 3]
@@ -125,7 +125,7 @@ Input [4]: [hd_demo_sk#13, hd_buy_potential#14, 
hd_dep_count#15, hd_vehicle_coun
 
 (20) Filter [codegen id : 3]
 Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, 
hd_vehicle_count#16]
-Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = 
>10000) OR (hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND 
(CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / 
cast(hd_vehicle_count#16 as double)) ELSE null END > 1.2)) AND 
isnotnull(hd_demo_sk#13))
+Condition : (((((isnotnull(hd_vehicle_count#16) AND 
isnotnull(hd_dep_count#15)) AND ((hd_buy_potential#14 = >10000) OR 
(hd_buy_potential#14 = unknown))) AND (hd_vehicle_count#16 > 0)) AND 
((cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) > 
1.2)) AND isnotnull(hd_demo_sk#13))
 
 (21) Project [codegen id : 3]
 Output [1]: [hd_demo_sk#13]
@@ -153,7 +153,7 @@ Results [3]: [ss_ticket_number#5, ss_customer_sk#2, 
count#19]
 
 (26) Exchange
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
-Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, 
[id=#20]
+Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), 
ENSURE_REQUIREMENTS, [id=#20]
 
 (27) HashAggregate [codegen id : 6]
 Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19]
@@ -195,7 +195,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, 
c_customer_sk#23, c_sa
 
 (35) Exchange
 Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, 
c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22]
-Arguments: rangepartitioning(c_last_name#26 ASC NULLS FIRST, c_first_name#25 
ASC NULLS FIRST, c_salutation#24 ASC NULLS FIRST, c_preferred_cust_flag#27 DESC 
NULLS LAST, ss_ticket_number#5 ASC NULLS FIRST, 5), true, [id=#29]
+Arguments: rangepartitioning(c_last_name#26 ASC NULLS FIRST, c_first_name#25 
ASC NULLS FIRST, c_salutation#24 ASC NULLS FIRST, c_preferred_cust_flag#27 DESC 
NULLS LAST, ss_ticket_number#5 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, 
[id=#29]
 
 (36) Sort [codegen id : 7]
 Input [6]: [c_last_name#26, c_first_name#25, c_salutation#24, 
c_preferred_cust_flag#27, ss_ticket_number#5, cnt#22]
diff --git 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt
 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt
index 8aa32fe..22cab3a 100644
--- 
a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt
+++ 
b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt
@@ -41,7 +41,7 @@ WholeStageCodegen (7)
                                 BroadcastExchange #5
                                   WholeStageCodegen (3)
                                     Project [hd_demo_sk]
-                                      Filter 
[hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+                                      Filter 
[hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk]
                                         ColumnarToRow
                                           InputAdapter
                                             Scan parquet 
default.household_demographics 
[hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to