From efe12aecf19804b92ad3e50aa78ad5f0e6c48c47 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sat, 3 Sep 2022 16:41:03 +0800 Subject: [PATCH] Revert "[SPARK-33861][SQL] Simplify conditional in predicate" This reverts commit 32d4a2b and 3aa4e11. Closes #37729 from wangyum/SPARK-33861. Authored-by: Yuming Wang Signed-off-by: Yuming Wang (cherry picked from commit 43cbdc6ec9dbcf9ebe0b48e14852cec4af18b4ec) Signed-off-by: Yuming Wang --- .../sql/catalyst/optimizer/Optimizer.scala | 1 - .../SimplifyConditionalsInPredicate.scala | 79 ----- .../sql/catalyst/rules/RuleIdCollection.scala | 1 - ...SimplifyConditionalsInPredicateSuite.scala | 249 --------------- .../q34.sf100/explain.txt | 4 +- .../q34.sf100/simplified.txt | 2 +- .../approved-plans-modified/q34/explain.txt | 4 +- .../q34/simplified.txt | 2 +- .../q53.sf100/explain.txt | 2 +- .../approved-plans-modified/q53/explain.txt | 2 +- .../q63.sf100/explain.txt | 2 +- .../approved-plans-modified/q63/explain.txt | 2 +- .../q73.sf100/explain.txt | 4 +- .../q73.sf100/simplified.txt | 2 +- .../approved-plans-modified/q73/explain.txt | 4 +- .../q73/simplified.txt | 2 +- .../q89.sf100/explain.txt | 2 +- .../approved-plans-modified/q89/explain.txt | 2 +- .../approved-plans-v1_4/q21.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q21/explain.txt | 2 +- .../approved-plans-v1_4/q34.sf100/explain.txt | 286 +++++++++--------- .../q34.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q34/explain.txt | 264 ++++++++-------- .../approved-plans-v1_4/q34/simplified.txt | 2 +- .../q39a.sf100/explain.txt | 4 +- .../q39a.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q39a/explain.txt | 4 +- .../approved-plans-v1_4/q39a/simplified.txt | 4 +- .../q39b.sf100/explain.txt | 4 +- .../q39b.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q39b/explain.txt | 4 +- .../approved-plans-v1_4/q39b/simplified.txt | 4 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q47/explain.txt | 2 +- .../approved-plans-v1_4/q53.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q53/explain.txt | 2 +- .../approved-plans-v1_4/q57.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q57/explain.txt | 2 +- .../approved-plans-v1_4/q63.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q63/explain.txt | 2 +- .../approved-plans-v1_4/q73.sf100/explain.txt | 4 +- .../q73.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q73/explain.txt | 4 +- .../approved-plans-v1_4/q73/simplified.txt | 2 +- .../approved-plans-v1_4/q89.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q89/explain.txt | 2 +- .../approved-plans-v2_7/q34.sf100/explain.txt | 4 +- .../q34.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q34/explain.txt | 4 +- .../approved-plans-v2_7/q34/simplified.txt | 2 +- .../approved-plans-v2_7/q47.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q47/explain.txt | 2 +- .../approved-plans-v2_7/q57.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q57/explain.txt | 2 +- 54 files changed, 343 insertions(+), 665 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicate.scala delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicateSuite.scala 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 6797bbfc71440..576628d689c66 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 @@ -113,7 +113,6 @@ 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 deleted file mode 100644 index c08bcbe8915bd..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicate.scala +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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, Coalesce, 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.catalyst.trees.TreePattern.{CASE_WHEN, IF} -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. This rule is null-safe. - * - * 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), falseVal) - * - IF(cond, true, falseVal) => OR(cond, falseVal) - * - 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 true ELSE elseVal END => OR(cond, elseVal) - */ -object SimplifyConditionalsInPredicate extends Rule[LogicalPlan] { - - def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning( - _.containsAnyPattern(CASE_WHEN, IF), ruleId) { - 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(Coalesce(Seq(cond, FalseLiteral))), trueValue) - case If(cond, FalseLiteral, falseValue) => - And(Not(Coalesce(Seq(cond, FalseLiteral))), 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(Coalesce(Seq(cond, FalseLiteral))), trueValue) - case CaseWhen(Seq((cond, FalseLiteral)), Some(elseValue)) => - And(Not(Coalesce(Seq(cond, FalseLiteral))), elseValue) - 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/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala index 2a05b8533bac1..267d84209c3cd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala @@ -148,7 +148,6 @@ object RuleIdCollection { "org.apache.spark.sql.catalyst.optimizer.SimplifyCaseConversionExpressions" :: "org.apache.spark.sql.catalyst.optimizer.SimplifyCasts" :: "org.apache.spark.sql.catalyst.optimizer.SimplifyConditionals" :: - "org.apache.spark.sql.catalyst.optimizer.SimplifyConditionalsInPredicate" :: "org.apache.spark.sql.catalyst.optimizer.SimplifyExtractValueOps" :: "org.apache.spark.sql.catalyst.optimizer.TransposeWindow" :: "org.apache.spark.sql.catalyst.optimizer.UnwrapCastInBinaryComparison" :: Nil 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 deleted file mode 100644 index 79db53e94f2c2..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicateSuite.scala +++ /dev/null @@ -1,249 +0,0 @@ -/* - * 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, Coalesce, Expression, If, IsNotNull, Literal, Not, Or, Rand} -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( - Not(Coalesce(Seq(UnresolvedAttribute("i") > Literal(10), FalseLiteral))), - 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( - Not(Coalesce(Seq(UnresolvedAttribute("i") > Literal(10), FalseLiteral))), - 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 = CaseWhen( - Seq((UnresolvedAttribute("i") > Literal(10), UnresolvedAttribute("b"))), - elseExp.filterNot(_.semanticEquals(Literal(null, BooleanType))))) - } - } - - 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( - Not(Coalesce(Seq(UnresolvedAttribute("i") > Literal(10), FalseLiteral))), - 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( - Not(Coalesce(Seq(UnresolvedAttribute("i") > Literal(10), FalseLiteral))), - 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 non-deterministic-cond THEN false END") { - val originalCond = - CaseWhen(Seq((UnresolvedAttribute("i") > Rand(0), FalseLiteral))) - val expectedCond = And(UnresolvedAttribute("i") > Rand(0), FalseLiteral) - // nondeterministic expressions are only allowed in Project, Filter, Aggregate or Window, - testFilter(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.as("out"), expectedExpr.as("out")) - } - - 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 bdd5dda489c61..8d77e0bb3e347 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 @@ -98,7 +98,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -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), IsNotNull(hd_demo_sk)] +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)] ReadSchema: struct (15) ColumnarToRow [codegen id : 3] @@ -106,7 +106,7 @@ Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_coun (16) Filter [codegen id : 3] Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : (((((isnotnull(hd_vehicle_count#13) AND isnotnull(hd_dep_count#12)) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = Unknown ))) AND (hd_vehicle_count#13 > 0)) AND ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2)) AND isnotnull(hd_demo_sk#10)) +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = Unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#10)) (17) Project [codegen id : 3] Output [1]: [hd_demo_sk#10] 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 1f4b2ff4eadd2..cb70bd42c1249 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 @@ -49,7 +49,7 @@ WholeStageCodegen (10) BroadcastExchange #6 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,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 dddd98f235cd1..6b79cf53e5dc1 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 @@ -95,7 +95,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -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), IsNotNull(hd_demo_sk)] +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)] ReadSchema: struct (15) ColumnarToRow [codegen id : 3] @@ -103,7 +103,7 @@ Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_coun (16) Filter [codegen id : 3] Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : (((((isnotnull(hd_vehicle_count#13) AND isnotnull(hd_dep_count#12)) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = Unknown ))) AND (hd_vehicle_count#13 > 0)) AND ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2)) AND isnotnull(hd_demo_sk#10)) +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = Unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#10)) (17) Project [codegen id : 3] Output [1]: [hd_demo_sk#10] 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 0e98ff2125b6b..1e88590181c6a 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 @@ -43,7 +43,7 @@ WholeStageCodegen (7) BroadcastExchange #5 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,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/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt index 8dae5f9eda3f0..8489daecb0290 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt @@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (26) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -Condition : (isnotnull(avg_quarterly_sales#23) AND ((avg_quarterly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) ELSE false END (27) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt index 6410e38de511a..88d3491f2ee2a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt @@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (26) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -Condition : (isnotnull(avg_quarterly_sales#23) AND ((avg_quarterly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) ELSE false END (27) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt index 6d0734c3e0d21..2fce7ea5c0cc5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt @@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowfram (26) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : (isnotnull(avg_monthly_sales#23) AND ((avg_monthly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) ELSE false END (27) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt index d7953e56df5ac..81873f4f1327d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt @@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowfram (26) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : (isnotnull(avg_monthly_sales#23) AND ((avg_monthly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) ELSE false END (27) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] 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 4cd58b1442653..fd5b51a120f22 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 @@ -95,7 +95,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -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), IsNotNull(hd_demo_sk)] +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)] ReadSchema: struct (15) ColumnarToRow [codegen id : 3] @@ -103,7 +103,7 @@ Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_coun (16) Filter [codegen id : 3] Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : (((((isnotnull(hd_vehicle_count#13) AND isnotnull(hd_dep_count#12)) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = Unknown ))) AND (hd_vehicle_count#13 > 0)) AND ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.0)) AND isnotnull(hd_demo_sk#10)) +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = Unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.0) END) AND isnotnull(hd_demo_sk#10)) (17) Project [codegen id : 3] Output [1]: [hd_demo_sk#10] 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 025e26182c30f..72653182711cf 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 @@ -46,7 +46,7 @@ WholeStageCodegen (7) BroadcastExchange #6 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,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 de22608640529..355612814c8d2 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 @@ -95,7 +95,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -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), IsNotNull(hd_demo_sk)] +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)] ReadSchema: struct (15) ColumnarToRow [codegen id : 3] @@ -103,7 +103,7 @@ Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_coun (16) Filter [codegen id : 3] Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : (((((isnotnull(hd_vehicle_count#13) AND isnotnull(hd_dep_count#12)) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = Unknown ))) AND (hd_vehicle_count#13 > 0)) AND ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.0)) AND isnotnull(hd_demo_sk#10)) +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = Unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.0) END) AND isnotnull(hd_demo_sk#10)) (17) Project [codegen id : 3] Output [1]: [hd_demo_sk#10] 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 311b101daa4a6..667bc0b2f4e93 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 @@ -43,7 +43,7 @@ WholeStageCodegen (7) BroadcastExchange #5 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,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/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt index b729963793e6f..9a4b6528b4164 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt @@ -141,7 +141,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#14, i_brand#12, s_store_ (25) Filter [codegen id : 7] Input [9]: [i_category#14, i_class#13, i_brand#12, s_store_name#9, s_company_name#10, d_moy#7, sum_sales#18, _w0#19, avg_monthly_sales#20] -Condition : (isnotnull(avg_monthly_sales#20) AND (NOT (avg_monthly_sales#20 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : CASE WHEN NOT (avg_monthly_sales#20 = 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) END (26) Project [codegen id : 7] Output [8]: [i_category#14, i_class#13, i_brand#12, s_store_name#9, s_company_name#10, d_moy#7, sum_sales#18, avg_monthly_sales#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt index 4471b2b86c69e..9a43760f24660 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt @@ -141,7 +141,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#4, i_brand#2, s_store_na (25) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] -Condition : (isnotnull(avg_monthly_sales#20) AND (NOT (avg_monthly_sales#20 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : CASE WHEN NOT (avg_monthly_sales#20 = 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) END (26) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt index abc5b2a95fa69..df074a5db136c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt @@ -130,7 +130,7 @@ Results [4]: [w_warehouse_name#12, i_item_id#7, sum(CASE WHEN (d_date#10 < 2000- (23) Filter [codegen id : 5] Input [4]: [w_warehouse_name#12, i_item_id#7, inv_before#19, inv_after#20] -Condition : ((isnotnull(inv_before#19) AND isnotnull(inv_after#20)) AND (((inv_before#19 > 0) AND ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) >= 0.666667)) AND ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) <= 1.5))) +Condition : (CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) >= 0.666667) END AND CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) <= 1.5) END) (24) TakeOrderedAndProject Input [4]: [w_warehouse_name#12, i_item_id#7, inv_before#19, inv_after#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt index 491d823d17a8a..b71195a424d7c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt @@ -130,7 +130,7 @@ Results [4]: [w_warehouse_name#7, i_item_id#9, sum(CASE WHEN (d_date#12 < 2000-0 (23) Filter [codegen id : 5] Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] -Condition : ((isnotnull(inv_before#19) AND isnotnull(inv_after#20)) AND (((inv_before#19 > 0) AND ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) >= 0.666667)) AND ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) <= 1.5))) +Condition : (CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) >= 0.666667) END AND CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) <= 1.5) END) (24) TakeOrderedAndProject Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] 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 ea9994910c3b9..fd71680298584 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 @@ -1,46 +1,42 @@ == Physical Plan == -* Sort (39) -+- Exchange (38) - +- * Project (37) - +- * SortMergeJoin Inner (36) - :- * Sort (30) - : +- Exchange (29) - : +- * Filter (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) +* Sort (35) ++- Exchange (34) + +- * Project (33) + +- * SortMergeJoin Inner (32) + :- * Sort (26) + : +- Exchange (25) + : +- * Filter (24) + : +- * HashAggregate (23) + : +- Exchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.store (11) - : +- BroadcastExchange (22) - : +- * Project (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.household_demographics (18) - +- * Sort (35) - +- Exchange (34) - +- * Filter (33) - +- * ColumnarToRow (32) - +- Scan parquet default.customer (31) - - -(1) Scan parquet default.store_sales + : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * Project (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * Project (17) + : +- * Filter (16) + : +- * ColumnarToRow (15) + : +- Scan parquet spark_catalog.default.household_demographics (14) + +- * Sort (31) + +- Exchange (30) + +- * Filter (29) + +- * ColumnarToRow (28) + +- Scan parquet spark_catalog.default.customer (27) + + +(1) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] @@ -55,175 +51,183 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(7) Project [codegen id : 1] +(4) ReusedExchange [Reuses operator id: 40] Output [1]: [d_date_sk#7] -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -(8) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] - -(9) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] +Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(6) Project [codegen id : 4] Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -(11) Scan parquet default.store -Output [2]: [s_store_sk#11, s_county#12] +(7) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#8, s_county#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_county#12] +(8) ColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#8, s_county#9] -(13) Filter [codegen id : 2] -Input [2]: [s_store_sk#11, s_county#12] -Condition : ((isnotnull(s_county#12) AND (s_county#12 = Williamson County)) AND isnotnull(s_store_sk#11)) +(9) Filter [codegen id : 2] +Input [2]: [s_store_sk#8, s_county#9] +Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) -(14) Project [codegen id : 2] -Output [1]: [s_store_sk#11] -Input [2]: [s_store_sk#11, s_county#12] +(10) Project [codegen id : 2] +Output [1]: [s_store_sk#8] +Input [2]: [s_store_sk#8, s_county#9] -(15) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +(11) BroadcastExchange +Input [1]: [s_store_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(16) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#11] +Right keys [1]: [s_store_sk#8] +Join type: Inner Join condition: None -(17) Project [codegen id : 4] +(13) Project [codegen id : 4] Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#11] +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] -(18) Scan parquet default.household_demographics -Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] +(14) Scan parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -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), IsNotNull(hd_demo_sk)] +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)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 3] -Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] +(15) ColumnarToRow [codegen id : 3] +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -(20) Filter [codegen id : 3] -Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +(16) Filter [codegen id : 3] +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#10)) -(21) Project [codegen id : 3] -Output [1]: [hd_demo_sk#14] -Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] +(17) Project [codegen id : 3] +Output [1]: [hd_demo_sk#10] +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -(22) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +(18) BroadcastExchange +Input [1]: [hd_demo_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#14] +Right keys [1]: [hd_demo_sk#10] +Join type: Inner Join condition: None -(24) Project [codegen id : 4] +(20) Project [codegen id : 4] Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#14] +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] -(25) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 4] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#19] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#20] +Aggregate Attributes [1]: [count#14] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(26) Exchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#20] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#21] +(22) Exchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(27) HashAggregate [codegen id : 5] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#20] +(23) HashAggregate [codegen id : 5] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#22] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#22 AS cnt#23] +Aggregate Attributes [1]: [count(1)#16] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(28) Filter [codegen id : 5] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#23] -Condition : ((cnt#23 >= 15) AND (cnt#23 <= 20)) +(24) Filter [codegen id : 5] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) -(29) Exchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#23] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#24] +(25) Exchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(30) Sort [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#23] +(26) Sort [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 -(31) Scan parquet default.customer -Output [5]: [c_customer_sk#25, c_salutation#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29] +(27) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 7] -Input [5]: [c_customer_sk#25, c_salutation#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29] +(28) ColumnarToRow [codegen id : 7] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(33) Filter [codegen id : 7] -Input [5]: [c_customer_sk#25, c_salutation#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29] -Condition : isnotnull(c_customer_sk#25) +(29) Filter [codegen id : 7] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Condition : isnotnull(c_customer_sk#18) -(34) Exchange -Input [5]: [c_customer_sk#25, c_salutation#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29] -Arguments: hashpartitioning(c_customer_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +(30) Exchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(35) Sort [codegen id : 8] -Input [5]: [c_customer_sk#25, c_salutation#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29] -Arguments: [c_customer_sk#25 ASC NULLS FIRST], false, 0 +(31) Sort [codegen id : 8] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [c_customer_sk#18 ASC NULLS FIRST], false, 0 -(36) SortMergeJoin [codegen id : 9] +(32) SortMergeJoin [codegen id : 9] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#25] +Right keys [1]: [c_customer_sk#18] +Join type: Inner Join condition: None -(37) Project [codegen id : 9] -Output [6]: [c_last_name#28, c_first_name#27, c_salutation#26, c_preferred_cust_flag#29, ss_ticket_number#4, cnt#23] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#23, c_customer_sk#25, c_salutation#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29] +(33) Project [codegen id : 9] +Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(38) Exchange -Input [6]: [c_last_name#28, c_first_name#27, c_salutation#26, c_preferred_cust_flag#29, ss_ticket_number#4, cnt#23] -Arguments: rangepartitioning(c_last_name#28 ASC NULLS FIRST, c_first_name#27 ASC NULLS FIRST, c_salutation#26 ASC NULLS FIRST, c_preferred_cust_flag#29 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [id=#31] +(34) Exchange +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(39) Sort [codegen id : 10] -Input [6]: [c_last_name#28, c_first_name#27, c_salutation#26, c_preferred_cust_flag#29, ss_ticket_number#4, cnt#23] -Arguments: [c_last_name#28 ASC NULLS FIRST, c_first_name#27 ASC NULLS FIRST, c_salutation#26 ASC NULLS FIRST, c_preferred_cust_flag#29 DESC NULLS LAST], true, 0 +(35) Sort [codegen id : 10] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (40) +BroadcastExchange (40) ++- * Project (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet spark_catalog.default.date_dim (36) + + +(36) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(37) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +(38) Filter [codegen id : 1] +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(40) ReusedExchange [Reuses operator id: 8] +(39) Project [codegen id : 1] Output [1]: [d_date_sk#7] +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] + +(40) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] 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 24c5c1c256c26..4e7e2b03c92c3 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 @@ -49,7 +49,7 @@ WholeStageCodegen (10) BroadcastExchange #6 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,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 df5b93da51771..c1945a13a2154 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 @@ -1,43 +1,39 @@ == Physical Plan == -* Sort (36) -+- Exchange (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Filter (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) +* Sort (32) ++- Exchange (31) + +- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * Filter (24) + : +- * HashAggregate (23) + : +- Exchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.store (11) - : +- BroadcastExchange (22) - : +- * Project (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.household_demographics (18) - +- BroadcastExchange (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet default.customer (29) - - -(1) Scan parquet default.store_sales + : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * Project (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * Project (17) + : +- * Filter (16) + : +- * ColumnarToRow (15) + : +- Scan parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (28) + +- * Filter (27) + +- * ColumnarToRow (26) + +- Scan parquet spark_catalog.default.customer (25) + + +(1) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] @@ -52,163 +48,171 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] - -(6) Filter [codegen id : 1] -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(7) Project [codegen id : 1] +(4) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#7] -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] - -(8) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] -(9) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] +Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(6) Project [codegen id : 4] Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -(11) Scan parquet default.store -Output [2]: [s_store_sk#11, s_county#12] +(7) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#8, s_county#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_county#12] +(8) ColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#8, s_county#9] -(13) Filter [codegen id : 2] -Input [2]: [s_store_sk#11, s_county#12] -Condition : ((isnotnull(s_county#12) AND (s_county#12 = Williamson County)) AND isnotnull(s_store_sk#11)) +(9) Filter [codegen id : 2] +Input [2]: [s_store_sk#8, s_county#9] +Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) -(14) Project [codegen id : 2] -Output [1]: [s_store_sk#11] -Input [2]: [s_store_sk#11, s_county#12] +(10) Project [codegen id : 2] +Output [1]: [s_store_sk#8] +Input [2]: [s_store_sk#8, s_county#9] -(15) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +(11) BroadcastExchange +Input [1]: [s_store_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(16) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#11] +Right keys [1]: [s_store_sk#8] +Join type: Inner Join condition: None -(17) Project [codegen id : 4] +(13) Project [codegen id : 4] Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#11] +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] -(18) Scan parquet default.household_demographics -Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] +(14) Scan parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -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), IsNotNull(hd_demo_sk)] +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)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 3] -Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] +(15) ColumnarToRow [codegen id : 3] +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -(20) Filter [codegen id : 3] -Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +(16) Filter [codegen id : 3] +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#10)) -(21) Project [codegen id : 3] -Output [1]: [hd_demo_sk#14] -Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] +(17) Project [codegen id : 3] +Output [1]: [hd_demo_sk#10] +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -(22) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +(18) BroadcastExchange +Input [1]: [hd_demo_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#14] +Right keys [1]: [hd_demo_sk#10] +Join type: Inner Join condition: None -(24) Project [codegen id : 4] +(20) Project [codegen id : 4] Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#14] +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] -(25) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 4] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#19] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#20] +Aggregate Attributes [1]: [count#14] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(26) Exchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#20] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#21] +(22) Exchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(27) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#20] +(23) HashAggregate [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#22] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#22 AS cnt#23] +Aggregate Attributes [1]: [count(1)#16] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(28) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#23] -Condition : ((cnt#23 >= 15) AND (cnt#23 <= 20)) +(24) Filter [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) -(29) Scan parquet default.customer -Output [5]: [c_customer_sk#24, c_salutation#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28] +(25) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#24, c_salutation#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28] +(26) ColumnarToRow [codegen id : 5] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(31) Filter [codegen id : 5] -Input [5]: [c_customer_sk#24, c_salutation#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28] -Condition : isnotnull(c_customer_sk#24) +(27) Filter [codegen id : 5] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Condition : isnotnull(c_customer_sk#18) -(32) BroadcastExchange -Input [5]: [c_customer_sk#24, c_salutation#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +(28) BroadcastExchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 6] +(29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#24] +Right keys [1]: [c_customer_sk#18] +Join type: Inner Join condition: None -(34) Project [codegen id : 6] -Output [6]: [c_last_name#27, c_first_name#26, c_salutation#25, c_preferred_cust_flag#28, ss_ticket_number#4, cnt#23] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#23, c_customer_sk#24, c_salutation#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28] +(30) Project [codegen id : 6] +Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(35) Exchange -Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, c_preferred_cust_flag#28, ss_ticket_number#4, cnt#23] -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] +(31) Exchange +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(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#4, cnt#23] -Arguments: [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], true, 0 +(32) Sort [codegen id : 7] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (37) +BroadcastExchange (37) ++- * Project (36) + +- * Filter (35) + +- * ColumnarToRow (34) + +- Scan parquet spark_catalog.default.date_dim (33) -(37) ReusedExchange [Reuses operator id: 8] +(33) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(34) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] + +(35) Filter [codegen id : 1] +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) + +(36) Project [codegen id : 1] Output [1]: [d_date_sk#7] +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] + +(37) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] 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 e4f89ab27658b..e47f447e46fe5 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 @@ -43,7 +43,7 @@ WholeStageCodegen (7) BroadcastExchange #5 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,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/q39a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt index 64b986d1a6fe4..8a1d86400e890 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt @@ -149,7 +149,7 @@ Results [5]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, stddev_samp(cast(inv_quant (22) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, stdev#23, mean#24] -Condition : ((isnotnull(stdev#23) AND isnotnull(mean#24)) AND (NOT coalesce((mean#24 = 0.0), false) AND ((stdev#23 / mean#24) > 1.0))) +Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END (23) Project [codegen id : 5] Output [5]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] @@ -234,7 +234,7 @@ Results [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stddev_samp(cast(inv_qu (41) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stdev#23, mean#24] -Condition : ((isnotnull(stdev#23) AND isnotnull(mean#24)) AND (NOT coalesce((mean#24 = 0.0), false) AND ((stdev#23 / mean#24) > 1.0))) +Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END (42) Project [codegen id : 11] Output [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt index 0c84e462edb97..7e4ffc89e4690 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt @@ -11,7 +11,7 @@ WholeStageCodegen (14) Exchange [i_item_sk,w_warehouse_sk] #2 WholeStageCodegen (5) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [stdev,mean] + Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #3 @@ -58,7 +58,7 @@ WholeStageCodegen (14) Exchange [i_item_sk,w_warehouse_sk] #7 WholeStageCodegen (11) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [stdev,mean] + Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index bbf9b5185e776..f58bc30f2a49b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -146,7 +146,7 @@ Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quan (22) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] -Condition : ((isnotnull(stdev#23) AND isnotnull(mean#24)) AND (NOT coalesce((mean#24 = 0.0), false) AND ((stdev#23 / mean#24) > 1.0))) +Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END (23) Project [codegen id : 10] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] @@ -223,7 +223,7 @@ Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_qu (39) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -Condition : ((isnotnull(stdev#23) AND isnotnull(mean#24)) AND (NOT coalesce((mean#24 = 0.0), false) AND ((stdev#23 / mean#24) > 1.0))) +Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END (40) Project [codegen id : 9] Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index 2c97e17e7e444..2cf9d5ea033af 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -5,7 +5,7 @@ WholeStageCodegen (11) WholeStageCodegen (10) BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [stdev,mean] + Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 @@ -49,7 +49,7 @@ WholeStageCodegen (11) BroadcastExchange #6 WholeStageCodegen (9) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [stdev,mean] + Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt index c9208985327a3..3fd1431555933 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt @@ -149,7 +149,7 @@ Results [5]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, stddev_samp(cast(inv_quant (22) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, stdev#23, mean#24] -Condition : ((isnotnull(stdev#23) AND isnotnull(mean#24)) AND ((NOT coalesce((mean#24 = 0.0), false) AND ((stdev#23 / mean#24) > 1.0)) AND ((stdev#23 / mean#24) > 1.5))) +Condition : (CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END AND CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.5) END) (23) Project [codegen id : 5] Output [5]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] @@ -234,7 +234,7 @@ Results [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stddev_samp(cast(inv_qu (41) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stdev#23, mean#24] -Condition : ((isnotnull(stdev#23) AND isnotnull(mean#24)) AND (NOT coalesce((mean#24 = 0.0), false) AND ((stdev#23 / mean#24) > 1.0))) +Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END (42) Project [codegen id : 11] Output [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt index 0c84e462edb97..7e4ffc89e4690 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt @@ -11,7 +11,7 @@ WholeStageCodegen (14) Exchange [i_item_sk,w_warehouse_sk] #2 WholeStageCodegen (5) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [stdev,mean] + Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #3 @@ -58,7 +58,7 @@ WholeStageCodegen (14) Exchange [i_item_sk,w_warehouse_sk] #7 WholeStageCodegen (11) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [stdev,mean] + Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index 3a8329c86102e..ae556d15fd1c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -146,7 +146,7 @@ Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quan (22) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] -Condition : ((isnotnull(stdev#23) AND isnotnull(mean#24)) AND ((NOT coalesce((mean#24 = 0.0), false) AND ((stdev#23 / mean#24) > 1.0)) AND ((stdev#23 / mean#24) > 1.5))) +Condition : (CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END AND CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.5) END) (23) Project [codegen id : 10] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] @@ -223,7 +223,7 @@ Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_qu (39) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -Condition : ((isnotnull(stdev#23) AND isnotnull(mean#24)) AND (NOT coalesce((mean#24 = 0.0), false) AND ((stdev#23 / mean#24) > 1.0))) +Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END (40) Project [codegen id : 9] Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index 2c97e17e7e444..2cf9d5ea033af 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -5,7 +5,7 @@ WholeStageCodegen (11) WholeStageCodegen (10) BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [stdev,mean] + Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 @@ -49,7 +49,7 @@ WholeStageCodegen (11) BroadcastExchange #6 WholeStageCodegen (9) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [stdev,mean] + Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt index fe5d96875e8b5..968ac1a31f190 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt @@ -186,7 +186,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#14, i_brand#13, s_store_ (30) Filter [codegen id : 11] Input [10]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) END) (31) Project [codegen id : 11] Output [9]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt index fad76b17b4847..46552f361bcdd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt @@ -167,7 +167,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_na (27) Filter [codegen id : 22] Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) END) (28) Project [codegen id : 22] Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt index 844d7420f66eb..e9abcf355218a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt @@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (26) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -Condition : (isnotnull(avg_quarterly_sales#23) AND ((avg_quarterly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) ELSE false END (27) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index f0aafc12343b3..3e45a2875697d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (26) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -Condition : (isnotnull(avg_quarterly_sales#23) AND ((avg_quarterly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) ELSE false END (27) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt index 4906d536aef85..0c81a3f3be482 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt @@ -186,7 +186,7 @@ Arguments: [avg(_w0#18) windowspecdefinition(i_category#13, i_brand#12, cc_name# (30) Filter [codegen id : 11] Input [9]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) END) (31) Project [codegen id : 11] Output [8]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt index 8711ea29b832b..7613e2d80ddda 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt @@ -167,7 +167,7 @@ Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13 (27) Filter [codegen id : 22] Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) END) (28) Project [codegen id : 22] Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt index cfe9e6ba99255..d7c7072935191 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt @@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowfram (26) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : (isnotnull(avg_monthly_sales#23) AND ((avg_monthly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) ELSE false END (27) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index 9e038c14d517e..17ea3f96a0e35 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowfram (26) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : (isnotnull(avg_monthly_sales#23) AND ((avg_monthly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) ELSE false END (27) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] 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 e538e2ce20a8c..2c9e15e5c0bb4 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 @@ -98,7 +98,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -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), IsNotNull(hd_demo_sk)] +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)] ReadSchema: struct (15) ColumnarToRow [codegen id : 3] @@ -106,7 +106,7 @@ Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_coun (16) Filter [codegen id : 3] Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : (((((isnotnull(hd_vehicle_count#13) AND isnotnull(hd_dep_count#12)) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.0)) AND isnotnull(hd_demo_sk#10)) +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.0) END) AND isnotnull(hd_demo_sk#10)) (17) Project [codegen id : 3] Output [1]: [hd_demo_sk#10] 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 33299bf87e5ab..2f3dc4ebef30f 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 @@ -49,7 +49,7 @@ WholeStageCodegen (10) BroadcastExchange #6 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,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 70e0397d07a0f..9c15e3f4d4343 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 @@ -95,7 +95,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -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), IsNotNull(hd_demo_sk)] +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)] ReadSchema: struct (15) ColumnarToRow [codegen id : 3] @@ -103,7 +103,7 @@ Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_coun (16) Filter [codegen id : 3] Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : (((((isnotnull(hd_vehicle_count#13) AND isnotnull(hd_dep_count#12)) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.0)) AND isnotnull(hd_demo_sk#10)) +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.0) END) AND isnotnull(hd_demo_sk#10)) (17) Project [codegen id : 3] Output [1]: [hd_demo_sk#10] 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 311b101daa4a6..667bc0b2f4e93 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 @@ -43,7 +43,7 @@ WholeStageCodegen (7) BroadcastExchange #5 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,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/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt index 5f0894579e00a..a17db9f6f5e26 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt @@ -141,7 +141,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#4, i_brand#2, s_store_na (25) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] -Condition : (isnotnull(avg_monthly_sales#20) AND (NOT (avg_monthly_sales#20 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : CASE WHEN NOT (avg_monthly_sales#20 = 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) END (26) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index b8b74b8baefe4..61a2c98de6067 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -141,7 +141,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#4, i_brand#2, s_store_na (25) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] -Condition : (isnotnull(avg_monthly_sales#20) AND (NOT (avg_monthly_sales#20 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : CASE WHEN NOT (avg_monthly_sales#20 = 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) END (26) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] 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 503bfdb0233cd..8e9ad05d3aeb1 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 @@ -98,7 +98,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -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), IsNotNull(hd_demo_sk)] +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)] ReadSchema: struct (15) ColumnarToRow [codegen id : 3] @@ -106,7 +106,7 @@ Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_coun (16) Filter [codegen id : 3] Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : (((((isnotnull(hd_vehicle_count#13) AND isnotnull(hd_dep_count#12)) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2)) AND isnotnull(hd_demo_sk#10)) +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#10)) (17) Project [codegen id : 3] Output [1]: [hd_demo_sk#10] 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 24279b0e0b3d5..0ce260795d12f 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 @@ -49,7 +49,7 @@ WholeStageCodegen (10) BroadcastExchange #6 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,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 41141558a807a..90497aab731d3 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 @@ -95,7 +95,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -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), IsNotNull(hd_demo_sk)] +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)] ReadSchema: struct (15) ColumnarToRow [codegen id : 3] @@ -103,7 +103,7 @@ Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_coun (16) Filter [codegen id : 3] Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : (((((isnotnull(hd_vehicle_count#13) AND isnotnull(hd_dep_count#12)) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2)) AND isnotnull(hd_demo_sk#10)) +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#10)) (17) Project [codegen id : 3] Output [1]: [hd_demo_sk#10] 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 4dd6079a7ecae..e9e68105865be 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 @@ -43,7 +43,7 @@ WholeStageCodegen (7) BroadcastExchange #5 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,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/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt index 1b637bcb84923..d8aafe9df514a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt @@ -186,7 +186,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#14, i_brand#13, s_store_ (30) Filter [codegen id : 11] Input [10]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) END) (31) Project [codegen id : 11] Output [9]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt index 26d1d7c0ced18..8ca983f3e1737 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt @@ -167,7 +167,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_na (27) Filter [codegen id : 22] Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) END) (28) Project [codegen id : 22] Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt index 34c177813c6de..642b1d2728b78 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt @@ -186,7 +186,7 @@ Arguments: [avg(_w0#18) windowspecdefinition(i_category#13, i_brand#12, cc_name# (30) Filter [codegen id : 11] Input [9]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) END) (31) Project [codegen id : 11] Output [8]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt index d209aae59b97e..7194fb2af8a05 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt @@ -167,7 +167,7 @@ Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13 (27) Filter [codegen id : 22] Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000) END) (28) Project [codegen id : 22] Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19]