From bbe2193442d14aa932c0e464fad4025def846e52 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 26 Jul 2021 23:19:57 +0800 Subject: [PATCH 01/17] Push down join condition evaluation --- .../PushDownJoinConditionEvaluation.scala | 57 ++ .../sql/catalyst/rules/RuleIdCollection.scala | 1 + ...PushDownJoinConditionEvaluationSuite.scala | 79 +++ .../spark/sql/execution/SparkOptimizer.scala | 5 +- .../q59.sf100/explain.txt | 12 +- .../q59.sf100/simplified.txt | 4 +- .../approved-plans-modified/q59/explain.txt | 12 +- .../q59/simplified.txt | 4 +- .../q65.sf100/explain.txt | 297 ++++++----- .../q65.sf100/simplified.txt | 49 +- .../approved-plans-modified/q65/explain.txt | 94 ++-- .../q65/simplified.txt | 4 +- .../approved-plans-v1_4/q1.sf100/explain.txt | 313 +++++------ .../q1.sf100/simplified.txt | 47 +- .../approved-plans-v1_4/q1/explain.txt | 295 +++++----- .../approved-plans-v1_4/q1/simplified.txt | 47 +- .../approved-plans-v1_4/q11.sf100/explain.txt | 88 +-- .../q11.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q11/explain.txt | 82 +-- .../approved-plans-v1_4/q11/simplified.txt | 4 +- .../approved-plans-v1_4/q15.sf100/explain.txt | 100 ++-- .../q15.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q15/explain.txt | 141 ++--- .../approved-plans-v1_4/q15/simplified.txt | 11 +- .../approved-plans-v1_4/q2.sf100/explain.txt | 12 +- .../q2.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q2/explain.txt | 12 +- .../approved-plans-v1_4/q2/simplified.txt | 4 +- .../q24a.sf100/explain.txt | 289 +++++----- .../q24a.sf100/simplified.txt | 17 +- .../approved-plans-v1_4/q24a/explain.txt | 362 +++++++------ .../approved-plans-v1_4/q24a/simplified.txt | 21 +- .../q24b.sf100/explain.txt | 289 +++++----- .../q24b.sf100/simplified.txt | 17 +- .../approved-plans-v1_4/q24b/explain.txt | 362 +++++++------ .../approved-plans-v1_4/q24b/simplified.txt | 21 +- .../approved-plans-v1_4/q30.sf100/explain.txt | 110 ++-- .../q30.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q30/explain.txt | 333 ++++++------ .../approved-plans-v1_4/q30/simplified.txt | 61 +-- .../approved-plans-v1_4/q31.sf100/explain.txt | 254 ++++----- .../q31.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q31/explain.txt | 473 +++++++++-------- .../approved-plans-v1_4/q31/simplified.txt | 175 +++--- .../approved-plans-v1_4/q32.sf100/explain.txt | 83 +-- .../q32.sf100/simplified.txt | 13 +- .../approved-plans-v1_4/q32/explain.txt | 88 +-- .../approved-plans-v1_4/q32/simplified.txt | 4 +- .../approved-plans-v1_4/q4.sf100/explain.txt | 262 ++++----- .../q4.sf100/simplified.txt | 6 +- .../approved-plans-v1_4/q4/explain.txt | 244 ++++----- .../approved-plans-v1_4/q4/simplified.txt | 6 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 48 +- .../q47.sf100/simplified.txt | 12 +- .../approved-plans-v1_4/q47/explain.txt | 40 +- .../approved-plans-v1_4/q47/simplified.txt | 4 +- .../approved-plans-v1_4/q57.sf100/explain.txt | 48 +- .../q57.sf100/simplified.txt | 12 +- .../approved-plans-v1_4/q57/explain.txt | 40 +- .../approved-plans-v1_4/q57/simplified.txt | 4 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 400 +++++++------- .../q58.sf100/simplified.txt | 182 ++++--- .../approved-plans-v1_4/q58/explain.txt | 395 +++++++------- .../approved-plans-v1_4/q58/simplified.txt | 167 +++--- .../approved-plans-v1_4/q59.sf100/explain.txt | 12 +- .../q59.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q59/explain.txt | 12 +- .../approved-plans-v1_4/q59/simplified.txt | 4 +- .../approved-plans-v1_4/q65.sf100/explain.txt | 321 +++++------ .../q65.sf100/simplified.txt | 49 +- .../approved-plans-v1_4/q65/explain.txt | 94 ++-- .../approved-plans-v1_4/q65/simplified.txt | 4 +- .../approved-plans-v1_4/q72.sf100/explain.txt | 502 +++++++++--------- .../q72.sf100/simplified.txt | 44 +- .../approved-plans-v1_4/q72/explain.txt | 92 ++-- .../approved-plans-v1_4/q72/simplified.txt | 4 +- .../approved-plans-v1_4/q8.sf100/explain.txt | 323 +++++------ .../q8.sf100/simplified.txt | 23 +- .../approved-plans-v1_4/q8/explain.txt | 283 +++++----- .../approved-plans-v1_4/q8/simplified.txt | 15 +- .../approved-plans-v1_4/q81.sf100/explain.txt | 110 ++-- .../q81.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q81/explain.txt | 325 ++++++------ .../approved-plans-v1_4/q81/simplified.txt | 61 +-- .../approved-plans-v1_4/q92.sf100/explain.txt | 101 ++-- .../q92.sf100/simplified.txt | 13 +- .../approved-plans-v1_4/q92/explain.txt | 98 ++-- .../approved-plans-v1_4/q92/simplified.txt | 2 +- .../approved-plans-v2_7/q11.sf100/explain.txt | 88 +-- .../q11.sf100/simplified.txt | 4 +- .../approved-plans-v2_7/q11/explain.txt | 82 +-- .../approved-plans-v2_7/q11/simplified.txt | 4 +- .../approved-plans-v2_7/q24.sf100/explain.txt | 476 ++++++++++------- .../q24.sf100/simplified.txt | 24 +- .../approved-plans-v2_7/q24/explain.txt | 376 +++++++------ .../approved-plans-v2_7/q24/simplified.txt | 21 +- .../approved-plans-v2_7/q47.sf100/explain.txt | 48 +- .../q47.sf100/simplified.txt | 12 +- .../approved-plans-v2_7/q47/explain.txt | 40 +- .../approved-plans-v2_7/q47/simplified.txt | 4 +- .../approved-plans-v2_7/q57.sf100/explain.txt | 48 +- .../q57.sf100/simplified.txt | 12 +- .../approved-plans-v2_7/q57/explain.txt | 40 +- .../approved-plans-v2_7/q57/simplified.txt | 4 +- .../approved-plans-v2_7/q72.sf100/explain.txt | 502 +++++++++--------- .../q72.sf100/simplified.txt | 44 +- .../approved-plans-v2_7/q72/explain.txt | 92 ++-- .../approved-plans-v2_7/q72/simplified.txt | 4 +- .../approved-plans-v2_7/q74.sf100/explain.txt | 88 +-- .../q74.sf100/simplified.txt | 4 +- .../approved-plans-v2_7/q74/explain.txt | 82 +-- .../approved-plans-v2_7/q74/simplified.txt | 4 +- 112 files changed, 5834 insertions(+), 5399 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluation.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluationSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluation.scala new file mode 100644 index 0000000000000..f439e02c86b81 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluation.scala @@ -0,0 +1,57 @@ +/* + * 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.{Alias, Expression, PredicateHelper} +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN + +/** + * Push down join condition evaluation to reduce eval expressions in join condition. + */ +object PushDownJoinConditionEvaluation extends Rule[LogicalPlan] with PredicateHelper { + def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning( + _.containsPattern(JOIN), ruleId) { + case j @ Join(left, right, _, Some(condition), _) => + val expressions = splitConjunctivePredicates(condition).flatMap(_.children).flatMap { + case e: Expression if e.children.nonEmpty => Seq(e) + case _ => Nil + } + + val leftKeys = expressions.filter(canEvaluate(_, left)) + val rightKeys = expressions.filter(canEvaluate(_, right)) + + val leftAlias = leftKeys.map(e => Alias(e, e.sql)()) + val rightAlias = rightKeys.map(e => Alias(e, e.sql)()) + + val newLeft = if (leftAlias.nonEmpty) Project(left.output ++ leftAlias, left) else left + val newRight = if (rightAlias.nonEmpty) Project(right.output ++ rightAlias, right) else right + + val map = leftKeys.zip(leftAlias).toMap ++ rightKeys.zip(rightAlias).toMap + val newCondition = if (leftAlias.nonEmpty || rightAlias.nonEmpty) { + condition.transformDown { + case e: Expression if e.references.nonEmpty && map.contains(e) => map(e).toAttribute + } + } else { + condition + } + + j.copy(left = newLeft, right = newRight, condition = Some(newCondition)) + } +} 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..b379d4d195457 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 @@ -144,6 +144,7 @@ object RuleIdCollection { "org.apache.spark.sql.catalyst.optimizer.ReplaceIntersectWithSemiJoin" :: "org.apache.spark.sql.catalyst.optimizer.RewriteExceptAll" :: "org.apache.spark.sql.catalyst.optimizer.RewriteIntersectAll" :: + "org.apache.spark.sql.catalyst.optimizer.PushDownJoinConditionEvaluation" :: "org.apache.spark.sql.catalyst.optimizer.SimplifyBinaryComparison" :: "org.apache.spark.sql.catalyst.optimizer.SimplifyCaseConversionExpressions" :: "org.apache.spark.sql.catalyst.optimizer.SimplifyCasts" :: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluationSuite.scala new file mode 100644 index 0000000000000..b3de7cf5a5ace --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluationSuite.scala @@ -0,0 +1,79 @@ +/* + * 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.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Alias, Coalesce, Substring, Upper} +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ + +class PushDownJoinConditionEvaluationSuite extends PlanTest { + + private object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Push down join condition evaluation", Once, + PushDownJoinConditionEvaluation, + CollapseProject) :: Nil + } + + private val testRelation = LocalRelation('a.string, 'b.int, 'c.int) + private val testRelation1 = LocalRelation('d.string, 'e.int) + private val x = testRelation.subquery('x) + private val y = testRelation1.subquery('y) + + test("Push down join condition evaluation(String expressions)") { + val joinType = Inner + Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => + val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) + val correctAnswer = + x.join(y.select("y.d".attr, "y.e".attr, Alias(udf, udf.sql)()), + joinType, Option("x.a".attr === s"`${udf.sql}`".attr)) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + } + + test("Push down join condition evaluation(null expressions)") { + val joinType = Inner + val udf = Coalesce(Seq("x.b".attr, "x.c".attr)) + val originalQuery = x.join(y, joinType, Option(udf === "y.e".attr)) + val correctAnswer = + x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join(y, + joinType, Option(s"`${udf.sql}`".attr === "y.e".attr)) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Negative case: all children are Attributes") { + val condition = Option("x.a".attr === "y.d".attr) + val originalQuery = x.join(y, Inner, condition) + val correctAnswer = x.join(y, Inner, condition) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Negative case: contains Literal") { + val condition = Option("x.a".attr === "string") + val originalQuery = x.join(y, Inner, condition) + val correctAnswer = x.join(y, Inner, condition) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index dde5dc2be0556..949ec7f75d2fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -48,7 +48,10 @@ class SparkOptimizer( PushDownPredicates) :+ Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, - PruneFilters)) ++ + PruneFilters) :+ + Batch("Push down join condition evaluation", Once, + PushDownJoinConditionEvaluation, + CollapseProject)) ++ postHocOptimizationBatches :+ Batch("Extract Python UDFs", Once, ExtractPythonUDFFromJoinCondition, diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt index d20f7cf4a2857..da204343a0d87 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt @@ -270,21 +270,21 @@ Right keys [1]: [d_week_seq#83] Join condition: None (47) Project [codegen id : 9] -Output [8]: [d_week_seq#58 AS d_week_seq2#85, s_store_id#80 AS s_store_id2#86, sun_sales#30 AS sun_sales2#87, mon_sales#31 AS mon_sales2#88, wed_sales#33 AS wed_sales2#89, thu_sales#34 AS thu_sales2#90, fri_sales#35 AS fri_sales2#91, sat_sales#36 AS sat_sales2#92] +Output [8]: [s_store_id#80 AS s_store_id2#85, sun_sales#30 AS sun_sales2#86, mon_sales#31 AS mon_sales2#87, wed_sales#33 AS wed_sales2#88, thu_sales#34 AS thu_sales2#89, fri_sales#35 AS fri_sales2#90, sat_sales#36 AS sat_sales2#91, (d_week_seq#58 - 52) AS (x.d_week_seq2 - 52)#92] Input [9]: [d_week_seq#58, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#80, d_week_seq#83] (48) BroadcastExchange -Input [8]: [d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, wed_sales2#89, thu_sales2#90, fri_sales2#91, sat_sales2#92] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#93] +Input [8]: [s_store_id2#85, sun_sales2#86, mon_sales2#87, wed_sales2#88, thu_sales2#89, fri_sales2#90, sat_sales2#91, (x.d_week_seq2 - 52)#92] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[7, int, true]),false), [id=#93] (49) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#86, (d_week_seq2#85 - 52)] +Right keys [2]: [s_store_id2#85, (x.d_week_seq2 - 52)#92] Join condition: None (50) Project [codegen id : 10] -Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#87)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#94, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#88)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#95, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#96, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#89)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#97, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#90)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#98, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#91)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#99, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#92)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#100] -Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, wed_sales2#89, thu_sales2#90, fri_sales2#91, sat_sales2#92] +Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#86)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#94, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#87)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#95, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#96, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#88)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#97, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#89)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#98, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#90)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#99, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#91)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#100] +Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, s_store_id2#85, sun_sales2#86, mon_sales2#87, wed_sales2#88, thu_sales2#89, fri_sales2#90, sat_sales2#91, (x.d_week_seq2 - 52)#92] (51) TakeOrderedAndProject Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#94, (mon_sales1 / mon_sales2)#95, (tue_sales1 / tue_sales1)#96, (wed_sales1 / wed_sales2)#97, (thu_sales1 / thu_sales2)#98, (fri_sales1 / fri_sales2)#99, (sat_sales1 / sat_sales2)#100] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt index eae7970dd89a0..604373b3caf7c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales1),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] WholeStageCodegen (10) Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] + BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,(x.d_week_seq2 - 52)] Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] @@ -42,7 +42,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Project [d_week_seq,s_store_id,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales] + Project [s_store_id,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt index d20f7cf4a2857..da204343a0d87 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt @@ -270,21 +270,21 @@ Right keys [1]: [d_week_seq#83] Join condition: None (47) Project [codegen id : 9] -Output [8]: [d_week_seq#58 AS d_week_seq2#85, s_store_id#80 AS s_store_id2#86, sun_sales#30 AS sun_sales2#87, mon_sales#31 AS mon_sales2#88, wed_sales#33 AS wed_sales2#89, thu_sales#34 AS thu_sales2#90, fri_sales#35 AS fri_sales2#91, sat_sales#36 AS sat_sales2#92] +Output [8]: [s_store_id#80 AS s_store_id2#85, sun_sales#30 AS sun_sales2#86, mon_sales#31 AS mon_sales2#87, wed_sales#33 AS wed_sales2#88, thu_sales#34 AS thu_sales2#89, fri_sales#35 AS fri_sales2#90, sat_sales#36 AS sat_sales2#91, (d_week_seq#58 - 52) AS (x.d_week_seq2 - 52)#92] Input [9]: [d_week_seq#58, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#80, d_week_seq#83] (48) BroadcastExchange -Input [8]: [d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, wed_sales2#89, thu_sales2#90, fri_sales2#91, sat_sales2#92] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#93] +Input [8]: [s_store_id2#85, sun_sales2#86, mon_sales2#87, wed_sales2#88, thu_sales2#89, fri_sales2#90, sat_sales2#91, (x.d_week_seq2 - 52)#92] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[7, int, true]),false), [id=#93] (49) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#86, (d_week_seq2#85 - 52)] +Right keys [2]: [s_store_id2#85, (x.d_week_seq2 - 52)#92] Join condition: None (50) Project [codegen id : 10] -Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#87)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#94, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#88)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#95, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#96, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#89)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#97, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#90)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#98, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#91)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#99, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#92)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#100] -Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, wed_sales2#89, thu_sales2#90, fri_sales2#91, sat_sales2#92] +Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#86)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#94, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#87)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#95, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#96, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#88)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#97, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#89)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#98, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#90)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#99, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#91)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#100] +Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, s_store_id2#85, sun_sales2#86, mon_sales2#87, wed_sales2#88, thu_sales2#89, fri_sales2#90, sat_sales2#91, (x.d_week_seq2 - 52)#92] (51) TakeOrderedAndProject Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#94, (mon_sales1 / mon_sales2)#95, (tue_sales1 / tue_sales1)#96, (wed_sales1 / wed_sales2)#97, (thu_sales1 / thu_sales2)#98, (fri_sales1 / fri_sales2)#99, (sat_sales1 / sat_sales2)#100] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/simplified.txt index eae7970dd89a0..604373b3caf7c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales1),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] WholeStageCodegen (10) Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] + BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,(x.d_week_seq2 - 52)] Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] @@ -42,7 +42,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Project [d_week_seq,s_store_id,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales] + Project [s_store_id,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt index 7066bd1ed142e..2de610eed5e3b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt @@ -1,42 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (38) -+- * Project (37) - +- * BroadcastHashJoin Inner BuildLeft (36) - :- BroadcastExchange (32) - : +- * Project (31) - : +- * BroadcastHashJoin Inner BuildLeft (30) - : :- BroadcastExchange (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Filter (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (23) - : : +- * HashAggregate (22) - : : +- Exchange (21) - : : +- * HashAggregate (20) - : : +- * HashAggregate (19) - : : +- Exchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.store_sales (11) - : : +- ReusedExchange (14) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet default.store (27) - +- * Filter (35) - +- * ColumnarToRow (34) - +- Scan parquet default.item (33) +TakeOrderedAndProject (39) ++- * Project (38) + +- * BroadcastHashJoin Inner BuildLeft (37) + :- BroadcastExchange (33) + : +- * Project (32) + : +- * BroadcastHashJoin Inner BuildLeft (31) + : :- BroadcastExchange (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (11) + : : : +- * Filter (10) + : : : +- * HashAggregate (9) + : : : +- Exchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (24) + : : +- * HashAggregate (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- * HashAggregate (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Filter (14) + : : : +- * ColumnarToRow (13) + : : : +- Scan parquet default.store_sales (12) + : : +- ReusedExchange (15) + : +- * Filter (30) + : +- * ColumnarToRow (29) + : +- Scan parquet default.store (28) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.item (34) (1) Scan parquet default.store_sales @@ -54,7 +55,7 @@ Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4] Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) -(4) ReusedExchange [Reuses operator id: 43] +(4) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 2] @@ -88,172 +89,176 @@ Results [3]: [ss_store_sk#2, ss_item_sk#1, MakeDecimal(sum(UnscaledValue(ss_sale Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] Condition : isnotnull(revenue#11) -(11) Scan parquet default.store_sales -Output [4]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +(11) Project [codegen id : 7] +Output [4]: [ss_store_sk#2, ss_item_sk#1, revenue#11, cast(revenue#11 as decimal(23,7)) AS CAST(sc.revenue AS DECIMAL(23,7))#12] +Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] + +(12) Scan parquet default.store_sales +Output [4]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#15), (ss_sold_date_sk#15 >= 2451911), (ss_sold_date_sk#15 <= 2452275), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ss_sold_date_sk#16), (ss_sold_date_sk#16 >= 2451911), (ss_sold_date_sk#16 <= 2452275), dynamicpruningexpression(ss_sold_date_sk#16 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [4]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +(13) ColumnarToRow [codegen id : 4] +Input [4]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16] -(13) Filter [codegen id : 4] -Input [4]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_store_sk#13) +(14) Filter [codegen id : 4] +Input [4]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16] +Condition : isnotnull(ss_store_sk#14) -(14) ReusedExchange [Reuses operator id: 43] -Output [1]: [d_date_sk#16] +(15) ReusedExchange [Reuses operator id: 44] +Output [1]: [d_date_sk#17] -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#15] -Right keys [1]: [d_date_sk#16] +(16) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#16] +Right keys [1]: [d_date_sk#17] Join condition: None -(16) Project [codegen id : 4] -Output [3]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14] -Input [5]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#16] - -(17) HashAggregate [codegen id : 4] -Input [3]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14] -Keys [2]: [ss_store_sk#13, ss_item_sk#12] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum#17] -Results [3]: [ss_store_sk#13, ss_item_sk#12, sum#18] +(17) Project [codegen id : 4] +Output [3]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15] +Input [5]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16, d_date_sk#17] -(18) Exchange -Input [3]: [ss_store_sk#13, ss_item_sk#12, sum#18] -Arguments: hashpartitioning(ss_store_sk#13, ss_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#19] +(18) HashAggregate [codegen id : 4] +Input [3]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15] +Keys [2]: [ss_store_sk#14, ss_item_sk#13] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#15))] +Aggregate Attributes [1]: [sum#18] +Results [3]: [ss_store_sk#14, ss_item_sk#13, sum#19] -(19) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#13, ss_item_sk#12, sum#18] -Keys [2]: [ss_store_sk#13, ss_item_sk#12] -Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] -Results [2]: [ss_store_sk#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS revenue#21] +(19) Exchange +Input [3]: [ss_store_sk#14, ss_item_sk#13, sum#19] +Arguments: hashpartitioning(ss_store_sk#14, ss_item_sk#13, 5), ENSURE_REQUIREMENTS, [id=#20] (20) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#13, revenue#21] -Keys [1]: [ss_store_sk#13] -Functions [1]: [partial_avg(revenue#21)] -Aggregate Attributes [2]: [sum#22, count#23] -Results [3]: [ss_store_sk#13, sum#24, count#25] - -(21) Exchange -Input [3]: [ss_store_sk#13, sum#24, count#25] -Arguments: hashpartitioning(ss_store_sk#13, 5), ENSURE_REQUIREMENTS, [id=#26] - -(22) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#13, sum#24, count#25] -Keys [1]: [ss_store_sk#13] -Functions [1]: [avg(revenue#21)] -Aggregate Attributes [1]: [avg(revenue#21)#27] -Results [2]: [ss_store_sk#13, avg(revenue#21)#27 AS ave#28] - -(23) BroadcastExchange -Input [2]: [ss_store_sk#13, ave#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] - -(24) BroadcastHashJoin [codegen id : 7] +Input [3]: [ss_store_sk#14, ss_item_sk#13, sum#19] +Keys [2]: [ss_store_sk#14, ss_item_sk#13] +Functions [1]: [sum(UnscaledValue(ss_sales_price#15))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#15))#21] +Results [2]: [ss_store_sk#14, MakeDecimal(sum(UnscaledValue(ss_sales_price#15))#21,17,2) AS revenue#22] + +(21) HashAggregate [codegen id : 5] +Input [2]: [ss_store_sk#14, revenue#22] +Keys [1]: [ss_store_sk#14] +Functions [1]: [partial_avg(revenue#22)] +Aggregate Attributes [2]: [sum#23, count#24] +Results [3]: [ss_store_sk#14, sum#25, count#26] + +(22) Exchange +Input [3]: [ss_store_sk#14, sum#25, count#26] +Arguments: hashpartitioning(ss_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#27] + +(23) HashAggregate [codegen id : 6] +Input [3]: [ss_store_sk#14, sum#25, count#26] +Keys [1]: [ss_store_sk#14] +Functions [1]: [avg(revenue#22)] +Aggregate Attributes [1]: [avg(revenue#22)#28] +Results [2]: [ss_store_sk#14, CheckOverflow((0.100000 * promote_precision(avg(revenue#22)#28)), DecimalType(23,7), true) AS (0.100000BD * sb.ave)#29] + +(24) BroadcastExchange +Input [2]: [ss_store_sk#14, (0.100000BD * sb.ave)#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] + +(25) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [ss_store_sk#13] -Join condition: (cast(revenue#11 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#28)), DecimalType(23,7), true)) +Right keys [1]: [ss_store_sk#14] +Join condition: (CAST(sc.revenue AS DECIMAL(23,7))#12 <= (0.100000BD * sb.ave)#29) -(25) Project [codegen id : 7] +(26) Project [codegen id : 7] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#11, ss_store_sk#13, ave#28] +Input [6]: [ss_store_sk#2, ss_item_sk#1, revenue#11, CAST(sc.revenue AS DECIMAL(23,7))#12, ss_store_sk#14, (0.100000BD * sb.ave)#29] -(26) BroadcastExchange +(27) BroadcastExchange Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] -(27) Scan parquet default.store -Output [2]: [s_store_sk#31, s_store_name#32] +(28) Scan parquet default.store +Output [2]: [s_store_sk#32, s_store_name#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(28) ColumnarToRow -Input [2]: [s_store_sk#31, s_store_name#32] +(29) ColumnarToRow +Input [2]: [s_store_sk#32, s_store_name#33] -(29) Filter -Input [2]: [s_store_sk#31, s_store_name#32] -Condition : isnotnull(s_store_sk#31) +(30) Filter +Input [2]: [s_store_sk#32, s_store_name#33] +Condition : isnotnull(s_store_sk#32) -(30) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#31] +Right keys [1]: [s_store_sk#32] Join condition: None -(31) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#11, s_store_name#32] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#11, s_store_sk#31, s_store_name#32] +(32) Project [codegen id : 8] +Output [3]: [ss_item_sk#1, revenue#11, s_store_name#33] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#11, s_store_sk#32, s_store_name#33] -(32) BroadcastExchange -Input [3]: [ss_item_sk#1, revenue#11, s_store_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] +(33) BroadcastExchange +Input [3]: [ss_item_sk#1, revenue#11, s_store_name#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] -(33) Scan parquet default.item -Output [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] +(34) Scan parquet default.item +Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) ColumnarToRow -Input [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] +(35) ColumnarToRow +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -(35) Filter -Input [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] -Condition : isnotnull(i_item_sk#34) +(36) Filter +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Condition : isnotnull(i_item_sk#35) -(36) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#34] +Right keys [1]: [i_item_sk#35] Join condition: None -(37) Project [codegen id : 9] -Output [6]: [s_store_name#32, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] -Input [8]: [ss_item_sk#1, revenue#11, s_store_name#32, i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] +(38) Project [codegen id : 9] +Output [6]: [s_store_name#33, i_item_desc#36, revenue#11, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [8]: [ss_item_sk#1, revenue#11, s_store_name#33, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -(38) TakeOrderedAndProject -Input [6]: [s_store_name#32, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] -Arguments: 100, [s_store_name#32 ASC NULLS FIRST, i_item_desc#35 ASC NULLS FIRST], [s_store_name#32, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] +(39) TakeOrderedAndProject +Input [6]: [s_store_name#33, i_item_desc#36, revenue#11, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Arguments: 100, [s_store_name#33 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#33, i_item_desc#36, revenue#11, i_current_price#37, i_wholesale_cost#38, i_brand#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (43) -+- * Project (42) - +- * Filter (41) - +- * ColumnarToRow (40) - +- Scan parquet default.date_dim (39) +BroadcastExchange (44) ++- * Project (43) + +- * Filter (42) + +- * ColumnarToRow (41) + +- Scan parquet default.date_dim (40) -(39) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#39] +(40) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), GreaterThanOrEqual(d_date_sk,2451911), LessThanOrEqual(d_date_sk,2452275), IsNotNull(d_date_sk)] ReadSchema: struct -(40) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#39] +(41) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#40] -(41) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#39] -Condition : (((((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1212)) AND (d_month_seq#39 <= 1223)) AND (d_date_sk#6 >= 2451911)) AND (d_date_sk#6 <= 2452275)) AND isnotnull(d_date_sk#6)) +(42) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#40] +Condition : (((((isnotnull(d_month_seq#40) AND (d_month_seq#40 >= 1212)) AND (d_month_seq#40 <= 1223)) AND (d_date_sk#6 >= 2451911)) AND (d_date_sk#6 <= 2452275)) AND isnotnull(d_date_sk#6)) -(42) Project [codegen id : 1] +(43) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_month_seq#39] +Input [2]: [d_date_sk#6, d_month_seq#40] -(43) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] -Subquery:2 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 12 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt index fd2cf0d33f543..ad18e1e3e80a3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt @@ -11,33 +11,34 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastExchange #2 WholeStageCodegen (7) Project [ss_store_sk,ss_item_sk,revenue] - BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] - Filter [revenue] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - InputAdapter - Exchange [ss_store_sk,ss_item_sk] #3 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 + BroadcastHashJoin [ss_store_sk,ss_store_sk,CAST(sc.revenue AS DECIMAL(23,7)),(0.100000BD * sb.ave)] + Project [ss_store_sk,ss_item_sk,revenue] + Filter [revenue] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] + InputAdapter + Exchange [ss_store_sk,ss_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #5 WholeStageCodegen (6) - HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),(0.100000BD * sb.ave),sum,count] InputAdapter Exchange [ss_store_sk] #6 WholeStageCodegen (5) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt index 02c9fdd520c10..5a1c7dab2b6fc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt @@ -139,84 +139,84 @@ Right keys [1]: [i_item_sk#15] Join condition: None (22) Project [codegen id : 9] -Output [7]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19] +Output [8]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19, cast(revenue#13 as decimal(23,7)) AS CAST(sc.revenue AS DECIMAL(23,7))#21] Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#13, i_item_sk#15, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19] (23) Scan parquet default.store_sales -Output [4]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24] +Output [4]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#24), (ss_sold_date_sk#24 >= 2451911), (ss_sold_date_sk#24 <= 2452275), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), (ss_sold_date_sk#25 >= 2451911), (ss_sold_date_sk#25 <= 2452275), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [4]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24] +Input [4]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25] (25) Filter [codegen id : 6] -Input [4]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24] -Condition : isnotnull(ss_store_sk#22) +Input [4]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_store_sk#23) (26) ReusedExchange [Reuses operator id: 43] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#26] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#24] -Right keys [1]: [d_date_sk#25] +Left keys [1]: [ss_sold_date_sk#25] +Right keys [1]: [d_date_sk#26] Join condition: None (28) Project [codegen id : 6] -Output [3]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23] -Input [5]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24, d_date_sk#25] +Output [3]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24] +Input [5]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#26] (29) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23] -Keys [2]: [ss_store_sk#22, ss_item_sk#21] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#23))] -Aggregate Attributes [1]: [sum#26] -Results [3]: [ss_store_sk#22, ss_item_sk#21, sum#27] +Input [3]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24] +Keys [2]: [ss_store_sk#23, ss_item_sk#22] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] +Aggregate Attributes [1]: [sum#27] +Results [3]: [ss_store_sk#23, ss_item_sk#22, sum#28] (30) Exchange -Input [3]: [ss_store_sk#22, ss_item_sk#21, sum#27] -Arguments: hashpartitioning(ss_store_sk#22, ss_item_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [ss_store_sk#23, ss_item_sk#22, sum#28] +Arguments: hashpartitioning(ss_store_sk#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [id=#29] (31) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#22, ss_item_sk#21, sum#27] -Keys [2]: [ss_store_sk#22, ss_item_sk#21] -Functions [1]: [sum(UnscaledValue(ss_sales_price#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#23))#29] -Results [2]: [ss_store_sk#22, MakeDecimal(sum(UnscaledValue(ss_sales_price#23))#29,17,2) AS revenue#30] +Input [3]: [ss_store_sk#23, ss_item_sk#22, sum#28] +Keys [2]: [ss_store_sk#23, ss_item_sk#22] +Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#24))#30] +Results [2]: [ss_store_sk#23, MakeDecimal(sum(UnscaledValue(ss_sales_price#24))#30,17,2) AS revenue#31] (32) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#22, revenue#30] -Keys [1]: [ss_store_sk#22] -Functions [1]: [partial_avg(revenue#30)] -Aggregate Attributes [2]: [sum#31, count#32] -Results [3]: [ss_store_sk#22, sum#33, count#34] +Input [2]: [ss_store_sk#23, revenue#31] +Keys [1]: [ss_store_sk#23] +Functions [1]: [partial_avg(revenue#31)] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ss_store_sk#23, sum#34, count#35] (33) Exchange -Input [3]: [ss_store_sk#22, sum#33, count#34] -Arguments: hashpartitioning(ss_store_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [ss_store_sk#23, sum#34, count#35] +Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, [id=#36] (34) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#22, sum#33, count#34] -Keys [1]: [ss_store_sk#22] -Functions [1]: [avg(revenue#30)] -Aggregate Attributes [1]: [avg(revenue#30)#36] -Results [2]: [ss_store_sk#22, avg(revenue#30)#36 AS ave#37] +Input [3]: [ss_store_sk#23, sum#34, count#35] +Keys [1]: [ss_store_sk#23] +Functions [1]: [avg(revenue#31)] +Aggregate Attributes [1]: [avg(revenue#31)#37] +Results [2]: [ss_store_sk#23, CheckOverflow((0.100000 * promote_precision(avg(revenue#31)#37)), DecimalType(23,7), true) AS (0.100000BD * sb.ave)#38] (35) BroadcastExchange -Input [2]: [ss_store_sk#22, ave#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] +Input [2]: [ss_store_sk#23, (0.100000BD * sb.ave)#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] -Right keys [1]: [ss_store_sk#22] -Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#37)), DecimalType(23,7), true)) +Right keys [1]: [ss_store_sk#23] +Join condition: (CAST(sc.revenue AS DECIMAL(23,7))#21 <= (0.100000BD * sb.ave)#38) (37) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19, ss_store_sk#22, ave#37] +Input [10]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19, CAST(sc.revenue AS DECIMAL(23,7))#21, ss_store_sk#23, (0.100000BD * sb.ave)#38] (38) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] @@ -233,27 +233,27 @@ BroadcastExchange (43) (39) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#39] +Output [2]: [d_date_sk#8, d_month_seq#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), GreaterThanOrEqual(d_date_sk,2451911), LessThanOrEqual(d_date_sk,2452275), IsNotNull(d_date_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#8, d_month_seq#39] +Input [2]: [d_date_sk#8, d_month_seq#40] (41) Filter [codegen id : 1] -Input [2]: [d_date_sk#8, d_month_seq#39] -Condition : (((((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1212)) AND (d_month_seq#39 <= 1223)) AND (d_date_sk#8 >= 2451911)) AND (d_date_sk#8 <= 2452275)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#8, d_month_seq#40] +Condition : (((((isnotnull(d_month_seq#40) AND (d_month_seq#40 >= 1212)) AND (d_month_seq#40 <= 1223)) AND (d_date_sk#8 >= 2451911)) AND (d_date_sk#8 <= 2452275)) AND isnotnull(d_date_sk#8)) (42) Project [codegen id : 1] Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_month_seq#39] +Input [2]: [d_date_sk#8, d_month_seq#40] (43) BroadcastExchange Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt index 4b519f37a58bf..e407c9cb608b5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] WholeStageCodegen (9) Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] + BroadcastHashJoin [ss_store_sk,ss_store_sk,CAST(sc.revenue AS DECIMAL(23,7)),(0.100000BD * sb.ave)] Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [s_store_name,ss_store_sk,ss_item_sk,revenue] @@ -45,7 +45,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes InputAdapter BroadcastExchange #5 WholeStageCodegen (8) - HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),(0.100000BD * sb.ave),sum,count] InputAdapter Exchange [ss_store_sk] #6 WholeStageCodegen (7) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt index f4fcd285bb57c..b2b74d02141cd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt @@ -1,47 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- * SortMergeJoin Inner (41) - :- * Sort (35) - : +- Exchange (34) - : +- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Filter (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_returns (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (24) - : : +- * Filter (23) - : : +- * HashAggregate (22) - : : +- Exchange (21) - : : +- * HashAggregate (20) - : : +- * HashAggregate (19) - : : +- Exchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.store_returns (11) - : : +- ReusedExchange (14) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet default.store (27) - +- * Sort (40) - +- Exchange (39) - +- * Filter (38) - +- * ColumnarToRow (37) - +- Scan parquet default.customer (36) +TakeOrderedAndProject (44) ++- * Project (43) + +- * SortMergeJoin Inner (42) + :- * Sort (36) + : +- Exchange (35) + : +- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (11) + : : : +- * Filter (10) + : : : +- * HashAggregate (9) + : : : +- Exchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_returns (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (25) + : : +- * Filter (24) + : : +- * HashAggregate (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- * HashAggregate (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Filter (14) + : : : +- * ColumnarToRow (13) + : : : +- Scan parquet default.store_returns (12) + : : +- ReusedExchange (15) + : +- BroadcastExchange (32) + : +- * Project (31) + : +- * Filter (30) + : +- * ColumnarToRow (29) + : +- Scan parquet default.store (28) + +- * Sort (41) + +- Exchange (40) + +- * Filter (39) + +- * ColumnarToRow (38) + +- Scan parquet default.customer (37) (1) Scan parquet default.store_returns @@ -59,7 +60,7 @@ Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_s Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) -(4) ReusedExchange [Reuses operator id: 48] +(4) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 2] @@ -93,192 +94,196 @@ Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#11, sr_store_sk#2 AS ctr_store Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13] Condition : isnotnull(ctr_total_return#13) -(11) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] +(11) Project [codegen id : 8] +Output [3]: [ctr_customer_sk#11, ctr_store_sk#12, cast(ctr_total_return#13 as decimal(24,7)) AS CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#14] +Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13] + +(12) Scan parquet default.store_returns +Output [4]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#17), dynamicpruningexpression(sr_returned_date_sk#17 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] +(13) ColumnarToRow [codegen id : 4] +Input [4]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18] -(13) Filter [codegen id : 4] -Input [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] -Condition : isnotnull(sr_store_sk#15) +(14) Filter [codegen id : 4] +Input [4]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18] +Condition : isnotnull(sr_store_sk#16) -(14) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#18] +(15) ReusedExchange [Reuses operator id: 49] +Output [1]: [d_date_sk#19] -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#17] -Right keys [1]: [d_date_sk#18] +(16) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [sr_returned_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None -(16) Project [codegen id : 4] -Output [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] -Input [5]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17, d_date_sk#18] +(17) Project [codegen id : 4] +Output [3]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17] +Input [5]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18, d_date_sk#19] -(17) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] -Keys [2]: [sr_customer_sk#14, sr_store_sk#15] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#16))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] +(18) HashAggregate [codegen id : 4] +Input [3]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17] +Keys [2]: [sr_customer_sk#15, sr_store_sk#16] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#17))] +Aggregate Attributes [1]: [sum#20] +Results [3]: [sr_customer_sk#15, sr_store_sk#16, sum#21] -(18) Exchange -Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] -Arguments: hashpartitioning(sr_customer_sk#14, sr_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#21] - -(19) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] -Keys [2]: [sr_customer_sk#14, sr_store_sk#15] -Functions [1]: [sum(UnscaledValue(sr_return_amt#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#16))#22] -Results [2]: [sr_store_sk#15 AS ctr_store_sk#12, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#22,17,2) AS ctr_total_return#13] +(19) Exchange +Input [3]: [sr_customer_sk#15, sr_store_sk#16, sum#21] +Arguments: hashpartitioning(sr_customer_sk#15, sr_store_sk#16, 5), ENSURE_REQUIREMENTS, [id=#22] (20) HashAggregate [codegen id : 5] +Input [3]: [sr_customer_sk#15, sr_store_sk#16, sum#21] +Keys [2]: [sr_customer_sk#15, sr_store_sk#16] +Functions [1]: [sum(UnscaledValue(sr_return_amt#17))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#17))#23] +Results [2]: [sr_store_sk#16 AS ctr_store_sk#12, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#23,17,2) AS ctr_total_return#13] + +(21) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#12, ctr_total_return#13] Keys [1]: [ctr_store_sk#12] Functions [1]: [partial_avg(ctr_total_return#13)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ctr_store_sk#12, sum#25, count#26] +Aggregate Attributes [2]: [sum#24, count#25] +Results [3]: [ctr_store_sk#12, sum#26, count#27] -(21) Exchange -Input [3]: [ctr_store_sk#12, sum#25, count#26] -Arguments: hashpartitioning(ctr_store_sk#12, 5), ENSURE_REQUIREMENTS, [id=#27] +(22) Exchange +Input [3]: [ctr_store_sk#12, sum#26, count#27] +Arguments: hashpartitioning(ctr_store_sk#12, 5), ENSURE_REQUIREMENTS, [id=#28] -(22) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#12, sum#25, count#26] +(23) HashAggregate [codegen id : 6] +Input [3]: [ctr_store_sk#12, sum#26, count#27] Keys [1]: [ctr_store_sk#12] Functions [1]: [avg(ctr_total_return#13)] -Aggregate Attributes [1]: [avg(ctr_total_return#13)#28] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#13)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12 AS ctr_store_sk#12#30] +Aggregate Attributes [1]: [avg(ctr_total_return#13)#29] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#13)#29) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12 AS ctr_store_sk#12#31] -(23) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) +(24) Filter [codegen id : 6] +Input [2]: [(avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12#31] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#30) -(24) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#31] +(25) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#32] -(25) BroadcastHashJoin [codegen id : 8] +(26) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#12] -Right keys [1]: [ctr_store_sk#12#30] -Join condition: (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) +Right keys [1]: [ctr_store_sk#12#31] +Join condition: (CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#14 > (avg(ctr_total_return) * 1.2)#30) -(26) Project [codegen id : 8] +(27) Project [codegen id : 8] Output [2]: [ctr_customer_sk#11, ctr_store_sk#12] -Input [5]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] +Input [5]: [ctr_customer_sk#11, ctr_store_sk#12, CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#14, (avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12#31] -(27) Scan parquet default.store -Output [2]: [s_store_sk#32, s_state#33] +(28) Scan parquet default.store +Output [2]: [s_store_sk#33, s_state#34] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#32, s_state#33] +(29) ColumnarToRow [codegen id : 7] +Input [2]: [s_store_sk#33, s_state#34] -(29) Filter [codegen id : 7] -Input [2]: [s_store_sk#32, s_state#33] -Condition : ((isnotnull(s_state#33) AND (s_state#33 = TN)) AND isnotnull(s_store_sk#32)) +(30) Filter [codegen id : 7] +Input [2]: [s_store_sk#33, s_state#34] +Condition : ((isnotnull(s_state#34) AND (s_state#34 = TN)) AND isnotnull(s_store_sk#33)) -(30) Project [codegen id : 7] -Output [1]: [s_store_sk#32] -Input [2]: [s_store_sk#32, s_state#33] +(31) Project [codegen id : 7] +Output [1]: [s_store_sk#33] +Input [2]: [s_store_sk#33, s_state#34] -(31) BroadcastExchange -Input [1]: [s_store_sk#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +(32) BroadcastExchange +Input [1]: [s_store_sk#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] -(32) BroadcastHashJoin [codegen id : 8] +(33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#12] -Right keys [1]: [s_store_sk#32] +Right keys [1]: [s_store_sk#33] Join condition: None -(33) Project [codegen id : 8] +(34) Project [codegen id : 8] Output [1]: [ctr_customer_sk#11] -Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#32] +Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#33] -(34) Exchange +(35) Exchange Input [1]: [ctr_customer_sk#11] -Arguments: hashpartitioning(ctr_customer_sk#11, 5), ENSURE_REQUIREMENTS, [id=#35] +Arguments: hashpartitioning(ctr_customer_sk#11, 5), ENSURE_REQUIREMENTS, [id=#36] -(35) Sort [codegen id : 9] +(36) Sort [codegen id : 9] Input [1]: [ctr_customer_sk#11] Arguments: [ctr_customer_sk#11 ASC NULLS FIRST], false, 0 -(36) Scan parquet default.customer -Output [2]: [c_customer_sk#36, c_customer_id#37] +(37) Scan parquet default.customer +Output [2]: [c_customer_sk#37, c_customer_id#38] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 10] -Input [2]: [c_customer_sk#36, c_customer_id#37] +(38) ColumnarToRow [codegen id : 10] +Input [2]: [c_customer_sk#37, c_customer_id#38] -(38) Filter [codegen id : 10] -Input [2]: [c_customer_sk#36, c_customer_id#37] -Condition : isnotnull(c_customer_sk#36) +(39) Filter [codegen id : 10] +Input [2]: [c_customer_sk#37, c_customer_id#38] +Condition : isnotnull(c_customer_sk#37) -(39) Exchange -Input [2]: [c_customer_sk#36, c_customer_id#37] -Arguments: hashpartitioning(c_customer_sk#36, 5), ENSURE_REQUIREMENTS, [id=#38] +(40) Exchange +Input [2]: [c_customer_sk#37, c_customer_id#38] +Arguments: hashpartitioning(c_customer_sk#37, 5), ENSURE_REQUIREMENTS, [id=#39] -(40) Sort [codegen id : 11] -Input [2]: [c_customer_sk#36, c_customer_id#37] -Arguments: [c_customer_sk#36 ASC NULLS FIRST], false, 0 +(41) Sort [codegen id : 11] +Input [2]: [c_customer_sk#37, c_customer_id#38] +Arguments: [c_customer_sk#37 ASC NULLS FIRST], false, 0 -(41) SortMergeJoin [codegen id : 12] +(42) SortMergeJoin [codegen id : 12] Left keys [1]: [ctr_customer_sk#11] -Right keys [1]: [c_customer_sk#36] +Right keys [1]: [c_customer_sk#37] Join condition: None -(42) Project [codegen id : 12] -Output [1]: [c_customer_id#37] -Input [3]: [ctr_customer_sk#11, c_customer_sk#36, c_customer_id#37] +(43) Project [codegen id : 12] +Output [1]: [c_customer_id#38] +Input [3]: [ctr_customer_sk#11, c_customer_sk#37, c_customer_id#38] -(43) TakeOrderedAndProject -Input [1]: [c_customer_id#37] -Arguments: 100, [c_customer_id#37 ASC NULLS FIRST], [c_customer_id#37] +(44) TakeOrderedAndProject +Input [1]: [c_customer_id#38] +Arguments: 100, [c_customer_id#38 ASC NULLS FIRST], [c_customer_id#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (48) -+- * Project (47) - +- * Filter (46) - +- * ColumnarToRow (45) - +- Scan parquet default.date_dim (44) +BroadcastExchange (49) ++- * Project (48) + +- * Filter (47) + +- * ColumnarToRow (46) + +- Scan parquet default.date_dim (45) -(44) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#39] +(45) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_year#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#39] +(46) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#40] -(46) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2000)) AND isnotnull(d_date_sk#6)) +(47) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#40] +Condition : ((isnotnull(d_year#40) AND (d_year#40 = 2000)) AND isnotnull(d_date_sk#6)) -(47) Project [codegen id : 1] +(48) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#39] +Input [2]: [d_date_sk#6, d_year#40] -(48) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] -Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#17 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 12 Hosting Expression = sr_returned_date_sk#18 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/simplified.txt index 08e121487a52b..4c1985607f1ad 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/simplified.txt @@ -11,29 +11,30 @@ TakeOrderedAndProject [c_customer_id] Project [ctr_customer_sk] BroadcastHashJoin [ctr_store_sk,s_store_sk] Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - InputAdapter - Exchange [sr_customer_sk,sr_store_sk] #2 - WholeStageCodegen (2) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk,sr_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastHashJoin [ctr_store_sk,ctr_store_sk,CAST(ctr1.ctr_total_return AS DECIMAL(24,7)),(avg(ctr_total_return) * 1.2)] + Project [ctr_customer_sk,ctr_store_sk,ctr_total_return] + Filter [ctr_total_return] + HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] + InputAdapter + Exchange [sr_customer_sk,sr_store_sk] #2 + WholeStageCodegen (2) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_store_sk,sr_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt index 5066d5c04fa50..a06086430a253 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt @@ -1,44 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Filter (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_returns (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (24) - : : +- * Filter (23) - : : +- * HashAggregate (22) - : : +- Exchange (21) - : : +- * HashAggregate (20) - : : +- * HashAggregate (19) - : : +- Exchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.store_returns (11) - : : +- ReusedExchange (14) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet default.store (27) - +- BroadcastExchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.customer (34) +TakeOrderedAndProject (41) ++- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (11) + : : : +- * Filter (10) + : : : +- * HashAggregate (9) + : : : +- Exchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_returns (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (25) + : : +- * Filter (24) + : : +- * HashAggregate (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- * HashAggregate (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Filter (14) + : : : +- * ColumnarToRow (13) + : : : +- Scan parquet default.store_returns (12) + : : +- ReusedExchange (15) + : +- BroadcastExchange (32) + : +- * Project (31) + : +- * Filter (30) + : +- * ColumnarToRow (29) + : +- Scan parquet default.store (28) + +- BroadcastExchange (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.customer (35) (1) Scan parquet default.store_returns @@ -56,7 +57,7 @@ Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_s Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) -(4) ReusedExchange [Reuses operator id: 45] +(4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 2] @@ -90,180 +91,184 @@ Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#11, sr_store_sk#2 AS ctr_store Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13] Condition : isnotnull(ctr_total_return#13) -(11) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] +(11) Project [codegen id : 9] +Output [3]: [ctr_customer_sk#11, ctr_store_sk#12, cast(ctr_total_return#13 as decimal(24,7)) AS CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#14] +Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13] + +(12) Scan parquet default.store_returns +Output [4]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#17), dynamicpruningexpression(sr_returned_date_sk#17 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] +(13) ColumnarToRow [codegen id : 4] +Input [4]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18] -(13) Filter [codegen id : 4] -Input [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] -Condition : isnotnull(sr_store_sk#15) +(14) Filter [codegen id : 4] +Input [4]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18] +Condition : isnotnull(sr_store_sk#16) -(14) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#18] +(15) ReusedExchange [Reuses operator id: 46] +Output [1]: [d_date_sk#19] -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#17] -Right keys [1]: [d_date_sk#18] +(16) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [sr_returned_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None -(16) Project [codegen id : 4] -Output [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] -Input [5]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17, d_date_sk#18] +(17) Project [codegen id : 4] +Output [3]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17] +Input [5]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18, d_date_sk#19] -(17) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] -Keys [2]: [sr_customer_sk#14, sr_store_sk#15] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#16))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] +(18) HashAggregate [codegen id : 4] +Input [3]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17] +Keys [2]: [sr_customer_sk#15, sr_store_sk#16] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#17))] +Aggregate Attributes [1]: [sum#20] +Results [3]: [sr_customer_sk#15, sr_store_sk#16, sum#21] -(18) Exchange -Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] -Arguments: hashpartitioning(sr_customer_sk#14, sr_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#21] - -(19) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] -Keys [2]: [sr_customer_sk#14, sr_store_sk#15] -Functions [1]: [sum(UnscaledValue(sr_return_amt#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#16))#22] -Results [2]: [sr_store_sk#15 AS ctr_store_sk#12, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#22,17,2) AS ctr_total_return#13] +(19) Exchange +Input [3]: [sr_customer_sk#15, sr_store_sk#16, sum#21] +Arguments: hashpartitioning(sr_customer_sk#15, sr_store_sk#16, 5), ENSURE_REQUIREMENTS, [id=#22] (20) HashAggregate [codegen id : 5] +Input [3]: [sr_customer_sk#15, sr_store_sk#16, sum#21] +Keys [2]: [sr_customer_sk#15, sr_store_sk#16] +Functions [1]: [sum(UnscaledValue(sr_return_amt#17))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#17))#23] +Results [2]: [sr_store_sk#16 AS ctr_store_sk#12, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#23,17,2) AS ctr_total_return#13] + +(21) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#12, ctr_total_return#13] Keys [1]: [ctr_store_sk#12] Functions [1]: [partial_avg(ctr_total_return#13)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ctr_store_sk#12, sum#25, count#26] +Aggregate Attributes [2]: [sum#24, count#25] +Results [3]: [ctr_store_sk#12, sum#26, count#27] -(21) Exchange -Input [3]: [ctr_store_sk#12, sum#25, count#26] -Arguments: hashpartitioning(ctr_store_sk#12, 5), ENSURE_REQUIREMENTS, [id=#27] +(22) Exchange +Input [3]: [ctr_store_sk#12, sum#26, count#27] +Arguments: hashpartitioning(ctr_store_sk#12, 5), ENSURE_REQUIREMENTS, [id=#28] -(22) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#12, sum#25, count#26] +(23) HashAggregate [codegen id : 6] +Input [3]: [ctr_store_sk#12, sum#26, count#27] Keys [1]: [ctr_store_sk#12] Functions [1]: [avg(ctr_total_return#13)] -Aggregate Attributes [1]: [avg(ctr_total_return#13)#28] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#13)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12 AS ctr_store_sk#12#30] +Aggregate Attributes [1]: [avg(ctr_total_return#13)#29] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#13)#29) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12 AS ctr_store_sk#12#31] -(23) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) +(24) Filter [codegen id : 6] +Input [2]: [(avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12#31] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#30) -(24) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#31] +(25) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#32] -(25) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#12] -Right keys [1]: [ctr_store_sk#12#30] -Join condition: (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) +Right keys [1]: [ctr_store_sk#12#31] +Join condition: (CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#14 > (avg(ctr_total_return) * 1.2)#30) -(26) Project [codegen id : 9] +(27) Project [codegen id : 9] Output [2]: [ctr_customer_sk#11, ctr_store_sk#12] -Input [5]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] +Input [5]: [ctr_customer_sk#11, ctr_store_sk#12, CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#14, (avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12#31] -(27) Scan parquet default.store -Output [2]: [s_store_sk#32, s_state#33] +(28) Scan parquet default.store +Output [2]: [s_store_sk#33, s_state#34] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#32, s_state#33] +(29) ColumnarToRow [codegen id : 7] +Input [2]: [s_store_sk#33, s_state#34] -(29) Filter [codegen id : 7] -Input [2]: [s_store_sk#32, s_state#33] -Condition : ((isnotnull(s_state#33) AND (s_state#33 = TN)) AND isnotnull(s_store_sk#32)) +(30) Filter [codegen id : 7] +Input [2]: [s_store_sk#33, s_state#34] +Condition : ((isnotnull(s_state#34) AND (s_state#34 = TN)) AND isnotnull(s_store_sk#33)) -(30) Project [codegen id : 7] -Output [1]: [s_store_sk#32] -Input [2]: [s_store_sk#32, s_state#33] +(31) Project [codegen id : 7] +Output [1]: [s_store_sk#33] +Input [2]: [s_store_sk#33, s_state#34] -(31) BroadcastExchange -Input [1]: [s_store_sk#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +(32) BroadcastExchange +Input [1]: [s_store_sk#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] -(32) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#12] -Right keys [1]: [s_store_sk#32] +Right keys [1]: [s_store_sk#33] Join condition: None -(33) Project [codegen id : 9] +(34) Project [codegen id : 9] Output [1]: [ctr_customer_sk#11] -Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#32] +Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#33] -(34) Scan parquet default.customer -Output [2]: [c_customer_sk#35, c_customer_id#36] +(35) Scan parquet default.customer +Output [2]: [c_customer_sk#36, c_customer_id#37] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#35, c_customer_id#36] +(36) ColumnarToRow [codegen id : 8] +Input [2]: [c_customer_sk#36, c_customer_id#37] -(36) Filter [codegen id : 8] -Input [2]: [c_customer_sk#35, c_customer_id#36] -Condition : isnotnull(c_customer_sk#35) +(37) Filter [codegen id : 8] +Input [2]: [c_customer_sk#36, c_customer_id#37] +Condition : isnotnull(c_customer_sk#36) -(37) BroadcastExchange -Input [2]: [c_customer_sk#35, c_customer_id#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] +(38) BroadcastExchange +Input [2]: [c_customer_sk#36, c_customer_id#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] -(38) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_customer_sk#11] -Right keys [1]: [c_customer_sk#35] +Right keys [1]: [c_customer_sk#36] Join condition: None -(39) Project [codegen id : 9] -Output [1]: [c_customer_id#36] -Input [3]: [ctr_customer_sk#11, c_customer_sk#35, c_customer_id#36] +(40) Project [codegen id : 9] +Output [1]: [c_customer_id#37] +Input [3]: [ctr_customer_sk#11, c_customer_sk#36, c_customer_id#37] -(40) TakeOrderedAndProject -Input [1]: [c_customer_id#36] -Arguments: 100, [c_customer_id#36 ASC NULLS FIRST], [c_customer_id#36] +(41) TakeOrderedAndProject +Input [1]: [c_customer_id#37] +Arguments: 100, [c_customer_id#37 ASC NULLS FIRST], [c_customer_id#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (45) -+- * Project (44) - +- * Filter (43) - +- * ColumnarToRow (42) - +- Scan parquet default.date_dim (41) +BroadcastExchange (46) ++- * Project (45) + +- * Filter (44) + +- * ColumnarToRow (43) + +- Scan parquet default.date_dim (42) -(41) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#38] +(42) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_year#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#38] +(43) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#39] -(43) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#38] -Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2000)) AND isnotnull(d_date_sk#6)) +(44) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2000)) AND isnotnull(d_date_sk#6)) -(44) Project [codegen id : 1] +(45) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#38] +Input [2]: [d_date_sk#6, d_year#39] -(45) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] -Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#17 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 12 Hosting Expression = sr_returned_date_sk#18 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt index f6fdeccf32fdf..362fe4c04804f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -5,29 +5,30 @@ TakeOrderedAndProject [c_customer_id] Project [ctr_customer_sk] BroadcastHashJoin [ctr_store_sk,s_store_sk] Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - InputAdapter - Exchange [sr_customer_sk,sr_store_sk] #1 - WholeStageCodegen (2) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk,sr_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastHashJoin [ctr_store_sk,ctr_store_sk,CAST(ctr1.ctr_total_return AS DECIMAL(24,7)),(avg(ctr_total_return) * 1.2)] + Project [ctr_customer_sk,ctr_store_sk,ctr_total_return] + Filter [ctr_total_return] + HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] + InputAdapter + Exchange [sr_customer_sk,sr_store_sk] #1 + WholeStageCodegen (2) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_store_sk,sr_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt index fa8839240f91e..fcc62c8dea526 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt @@ -355,94 +355,94 @@ Right keys [1]: [customer_id#70] Join condition: None (61) Project [codegen id : 26] -Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#71] +Output [4]: [customer_id#22, customer_preferred_cust_flag#46, year_total#71, CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#73] Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, customer_id#70, year_total#71] (62) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] +Output [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#76), dynamicpruningexpression(ws_sold_date_sk#76 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_sold_date_sk#77 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (63) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] +Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77] (64) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] -Condition : isnotnull(ws_bill_customer_sk#73) +Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77] +Condition : isnotnull(ws_bill_customer_sk#74) (65) ReusedExchange [Reuses operator id: 89] -Output [2]: [d_date_sk#77, d_year#78] +Output [2]: [d_date_sk#78, d_year#79] (66) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#76] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [ws_sold_date_sk#77] +Right keys [1]: [d_date_sk#78] Join condition: None (67) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Input [6]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76, d_date_sk#77, d_year#78] +Output [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77, d_date_sk#78, d_year#79] (68) Exchange -Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Arguments: hashpartitioning(ws_bill_customer_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] +Arguments: hashpartitioning(ws_bill_customer_sk#74, 5), ENSURE_REQUIREMENTS, [id=#80] (69) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Arguments: [ws_bill_customer_sk#73 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] +Arguments: [ws_bill_customer_sk#74 ASC NULLS FIRST], false, 0 (70) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] +Output [8]: [c_customer_sk#81, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88] (71) Sort [codegen id : 31] -Input [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] -Arguments: [c_customer_sk#80 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#81, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88] +Arguments: [c_customer_sk#81 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#73] -Right keys [1]: [c_customer_sk#80] +Left keys [1]: [ws_bill_customer_sk#74] +Right keys [1]: [c_customer_sk#81] Join condition: None (73) Project [codegen id : 32] -Output [10]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Input [12]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78, c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] +Output [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] +Input [12]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79, c_customer_sk#81, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88] (74) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Keys [8]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#88] -Results [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] +Input [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] +Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#89] +Results [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, sum#90] (75) Exchange -Input [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] -Arguments: hashpartitioning(c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, sum#90] +Arguments: hashpartitioning(c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, 5), ENSURE_REQUIREMENTS, [id=#91] (76) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] -Keys [8]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))#91] -Results [2]: [c_customer_id#81 AS customer_id#92, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))#91,18,2) AS year_total#93] +Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, sum#90] +Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))#92] +Results [2]: [c_customer_id#82 AS customer_id#93, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))#92,18,2) AS year_total#94] (77) Exchange -Input [2]: [customer_id#92, year_total#93] -Arguments: hashpartitioning(customer_id#92, 5), ENSURE_REQUIREMENTS, [id=#94] +Input [2]: [customer_id#93, year_total#94] +Arguments: hashpartitioning(customer_id#93, 5), ENSURE_REQUIREMENTS, [id=#95] (78) Sort [codegen id : 34] -Input [2]: [customer_id#92, year_total#93] -Arguments: [customer_id#92 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#93, year_total#94] +Arguments: [customer_id#93 ASC NULLS FIRST], false, 0 (79) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#92] -Join condition: (CASE WHEN (year_total#71 > 0.00) THEN CheckOverflow((promote_precision(year_total#93) / promote_precision(year_total#71)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END) +Right keys [1]: [customer_id#93] +Join condition: (CASE WHEN (year_total#71 > 0.00) THEN CheckOverflow((promote_precision(year_total#94) / promote_precision(year_total#71)), DecimalType(38,20), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#73) (80) Project [codegen id : 35] Output [1]: [customer_preferred_cust_flag#46] -Input [7]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#71, customer_id#92, year_total#93] +Input [6]: [customer_id#22, customer_preferred_cust_flag#46, year_total#71, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#73, customer_id#93, year_total#94] (81) TakeOrderedAndProject Input [1]: [customer_preferred_cust_flag#46] @@ -473,7 +473,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2001)) AND isnotnull(d_date_sk (85) BroadcastExchange Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (89) @@ -498,10 +498,10 @@ Condition : ((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND isnotnull(d_date_ (89) BroadcastExchange Input [2]: [d_date_sk#30, d_year#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#97] Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#76 IN dynamicpruning#29 +Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#29 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt index 889fc666bd810..7162fb28da5ce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (35) Project [customer_preferred_cust_flag] - SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END] InputAdapter WholeStageCodegen (26) - Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] + Project [customer_id,customer_preferred_cust_flag,year_total,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt index 3f9a0cb61ea1a..264122a9ef09e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt @@ -325,93 +325,93 @@ Right keys [1]: [customer_id#68] Join condition: None (54) Project [codegen id : 16] -Output [5]: [customer_id#21, year_total#22, customer_preferred_cust_flag#44, year_total#45, year_total#69] +Output [4]: [customer_id#21, customer_preferred_cust_flag#44, year_total#69, CASE WHEN (year_total#22 > 0.00) THEN CheckOverflow((promote_precision(year_total#45) / promote_precision(year_total#22)), DecimalType(38,20), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#71] Input [6]: [customer_id#21, year_total#22, customer_preferred_cust_flag#44, year_total#45, customer_id#68, year_total#69] (55) Scan parquet default.customer -Output [8]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78] +Output [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (56) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78] +Input [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] (57) Filter [codegen id : 14] -Input [8]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78] -Condition : (isnotnull(c_customer_sk#71) AND isnotnull(c_customer_id#72)) +Input [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] +Condition : (isnotnull(c_customer_sk#72) AND isnotnull(c_customer_id#73)) (58) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] +Output [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#82), dynamicpruningexpression(ws_sold_date_sk#82 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#35)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] +Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] (60) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] -Condition : isnotnull(ws_bill_customer_sk#79) +Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] +Condition : isnotnull(ws_bill_customer_sk#80) (61) BroadcastExchange -Input [4]: [ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#83] +Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#84] (62) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#71] -Right keys [1]: [ws_bill_customer_sk#79] +Left keys [1]: [c_customer_sk#72] +Right keys [1]: [ws_bill_customer_sk#80] Join condition: None (63) Project [codegen id : 14] -Output [10]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] -Input [12]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] +Output [10]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] +Input [12]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] (64) ReusedExchange [Reuses operator id: 81] -Output [2]: [d_date_sk#84, d_year#85] +Output [2]: [d_date_sk#85, d_year#86] (65) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#82] -Right keys [1]: [d_date_sk#84] +Left keys [1]: [ws_sold_date_sk#83] +Right keys [1]: [d_date_sk#85] Join condition: None (66) Project [codegen id : 14] -Output [10]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_ext_discount_amt#80, ws_ext_list_price#81, d_year#85] -Input [12]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82, d_date_sk#84, d_year#85] +Output [10]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, ws_ext_discount_amt#81, ws_ext_list_price#82, d_year#86] +Input [12]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83, d_date_sk#85, d_year#86] (67) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_ext_discount_amt#80, ws_ext_list_price#81, d_year#85] -Keys [8]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#80 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#86] -Results [9]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85, sum#87] +Input [10]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, ws_ext_discount_amt#81, ws_ext_list_price#82, d_year#86] +Keys [8]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#86] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#82 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#81 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#87] +Results [9]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#86, sum#88] (68) Exchange -Input [9]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85, sum#87] -Arguments: hashpartitioning(c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [9]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#86, sum#88] +Arguments: hashpartitioning(c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#86, 5), ENSURE_REQUIREMENTS, [id=#89] (69) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85, sum#87] -Keys [8]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#80 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#80 as decimal(8,2)))), DecimalType(8,2), true)))#89] -Results [2]: [c_customer_id#72 AS customer_id#90, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#80 as decimal(8,2)))), DecimalType(8,2), true)))#89,18,2) AS year_total#91] +Input [9]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#86, sum#88] +Keys [8]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#86] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#82 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#81 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#82 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#81 as decimal(8,2)))), DecimalType(8,2), true)))#90] +Results [2]: [c_customer_id#73 AS customer_id#91, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#82 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#81 as decimal(8,2)))), DecimalType(8,2), true)))#90,18,2) AS year_total#92] (70) BroadcastExchange -Input [2]: [customer_id#90, year_total#91] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#92] +Input [2]: [customer_id#91, year_total#92] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#93] (71) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#21] -Right keys [1]: [customer_id#90] -Join condition: (CASE WHEN (year_total#69 > 0.00) THEN CheckOverflow((promote_precision(year_total#91) / promote_precision(year_total#69)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#22 > 0.00) THEN CheckOverflow((promote_precision(year_total#45) / promote_precision(year_total#22)), DecimalType(38,20), true) ELSE null END) +Right keys [1]: [customer_id#91] +Join condition: (CASE WHEN (year_total#69 > 0.00) THEN CheckOverflow((promote_precision(year_total#92) / promote_precision(year_total#69)), DecimalType(38,20), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#71) (72) Project [codegen id : 16] Output [1]: [customer_preferred_cust_flag#44] -Input [7]: [customer_id#21, year_total#22, customer_preferred_cust_flag#44, year_total#45, year_total#69, customer_id#90, year_total#91] +Input [6]: [customer_id#21, customer_preferred_cust_flag#44, year_total#69, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#71, customer_id#91, year_total#92] (73) TakeOrderedAndProject Input [1]: [customer_preferred_cust_flag#44] @@ -442,7 +442,7 @@ Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2001)) AND isnotnull(d_date_ (77) BroadcastExchange Input [2]: [d_date_sk#15, d_year#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#93] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#94] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#34 IN dynamicpruning#35 BroadcastExchange (81) @@ -467,10 +467,10 @@ Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_ (81) BroadcastExchange Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#94] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#82 IN dynamicpruning#35 +Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#35 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index f860a9b8280a8..394355ed3a4c6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -1,8 +1,8 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (16) Project [customer_preferred_cust_flag] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END] + Project [customer_id,customer_preferred_cust_flag,year_total,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] Project [customer_id,year_total,customer_preferred_cust_flag,year_total] BroadcastHashJoin [customer_id,customer_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt index 55c39ed2e3288..6be8325f6af7b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt @@ -53,108 +53,108 @@ Right keys [1]: [d_date_sk#5] Join condition: None (6) Project [codegen id : 2] -Output [2]: [cs_bill_customer_sk#1, cs_sales_price#2] +Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, (cs_sales_price#2 > 500.00) AS (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6] Input [4]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, d_date_sk#5] (7) Exchange -Input [2]: [cs_bill_customer_sk#1, cs_sales_price#2] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#6] +Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#7] (8) Sort [codegen id : 3] -Input [2]: [cs_bill_customer_sk#1, cs_sales_price#2] +Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (9) Scan parquet default.customer -Output [2]: [c_customer_sk#7, c_current_addr_sk#8] +Output [2]: [c_customer_sk#8, c_current_addr_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 4] -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] +Input [2]: [c_customer_sk#8, c_current_addr_sk#9] (11) Filter [codegen id : 4] -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) +Input [2]: [c_customer_sk#8, c_current_addr_sk#9] +Condition : (isnotnull(c_customer_sk#8) AND isnotnull(c_current_addr_sk#9)) (12) Exchange -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: hashpartitioning(c_current_addr_sk#8, 5), ENSURE_REQUIREMENTS, [id=#9] +Input [2]: [c_customer_sk#8, c_current_addr_sk#9] +Arguments: hashpartitioning(c_current_addr_sk#9, 5), ENSURE_REQUIREMENTS, [id=#10] (13) Sort [codegen id : 5] -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [c_current_addr_sk#8 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#8, c_current_addr_sk#9] +Arguments: [c_current_addr_sk#9 ASC NULLS FIRST], false, 0 (14) Scan parquet default.customer_address -Output [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] +Output [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 6] -Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] +Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] (16) Filter [codegen id : 6] -Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] -Condition : isnotnull(ca_address_sk#10) +Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] +Condition : isnotnull(ca_address_sk#11) (17) Exchange -Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] -Arguments: hashpartitioning(ca_address_sk#10, 5), ENSURE_REQUIREMENTS, [id=#13] +Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] +Arguments: hashpartitioning(ca_address_sk#11, 5), ENSURE_REQUIREMENTS, [id=#14] (18) Sort [codegen id : 7] -Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] -Arguments: [ca_address_sk#10 ASC NULLS FIRST], false, 0 +Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] +Arguments: [ca_address_sk#11 ASC NULLS FIRST], false, 0 (19) SortMergeJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#8] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [c_current_addr_sk#9] +Right keys [1]: [ca_address_sk#11] Join condition: None (20) Project [codegen id : 8] -Output [3]: [c_customer_sk#7, ca_state#11, ca_zip#12] -Input [5]: [c_customer_sk#7, c_current_addr_sk#8, ca_address_sk#10, ca_state#11, ca_zip#12] +Output [3]: [c_customer_sk#8, ca_zip#13, (substr(ca_zip#13, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#12 IN (CA,WA,GA)) AS ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] +Input [5]: [c_customer_sk#8, c_current_addr_sk#9, ca_address_sk#11, ca_state#12, ca_zip#13] (21) Exchange -Input [3]: [c_customer_sk#7, ca_state#11, ca_zip#12] -Arguments: hashpartitioning(c_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [3]: [c_customer_sk#8, ca_zip#13, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] +Arguments: hashpartitioning(c_customer_sk#8, 5), ENSURE_REQUIREMENTS, [id=#16] (22) Sort [codegen id : 9] -Input [3]: [c_customer_sk#7, ca_state#11, ca_zip#12] -Arguments: [c_customer_sk#7 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#8, ca_zip#13, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] +Arguments: [c_customer_sk#8 ASC NULLS FIRST], false, 0 (23) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#7] -Join condition: ((substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#11 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) +Right keys [1]: [c_customer_sk#8] +Join condition: (((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15 OR (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6) (24) Project [codegen id : 10] -Output [2]: [cs_sales_price#2, ca_zip#12] -Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, c_customer_sk#7, ca_state#11, ca_zip#12] +Output [2]: [cs_sales_price#2, ca_zip#13] +Input [6]: [cs_bill_customer_sk#1, cs_sales_price#2, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6, c_customer_sk#8, ca_zip#13, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] (25) HashAggregate [codegen id : 10] -Input [2]: [cs_sales_price#2, ca_zip#12] -Keys [1]: [ca_zip#12] +Input [2]: [cs_sales_price#2, ca_zip#13] +Keys [1]: [ca_zip#13] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [ca_zip#12, sum#16] +Aggregate Attributes [1]: [sum#17] +Results [2]: [ca_zip#13, sum#18] (26) Exchange -Input [2]: [ca_zip#12, sum#16] -Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [2]: [ca_zip#13, sum#18] +Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, [id=#19] (27) HashAggregate [codegen id : 11] -Input [2]: [ca_zip#12, sum#16] -Keys [1]: [ca_zip#12] +Input [2]: [ca_zip#13, sum#18] +Keys [1]: [ca_zip#13] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#18] -Results [2]: [ca_zip#12, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#18,17,2) AS sum(cs_sales_price)#19] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#20] +Results [2]: [ca_zip#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#20,17,2) AS sum(cs_sales_price)#21] (28) TakeOrderedAndProject -Input [2]: [ca_zip#12, sum(cs_sales_price)#19] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST], [ca_zip#12, sum(cs_sales_price)#19] +Input [2]: [ca_zip#13, sum(cs_sales_price)#21] +Arguments: 100, [ca_zip#13 ASC NULLS FIRST], [ca_zip#13, sum(cs_sales_price)#21] ===== Subqueries ===== @@ -167,25 +167,25 @@ BroadcastExchange (33) (29) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#20, d_qoy#21] +Output [3]: [d_date_sk#5, d_year#22, d_qoy#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#20, d_qoy#21] +Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] (31) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#20, d_qoy#21] -Condition : ((((isnotnull(d_qoy#21) AND isnotnull(d_year#20)) AND (d_qoy#21 = 2)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] +Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 2)) AND (d_year#22 = 2001)) AND isnotnull(d_date_sk#5)) (32) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#20, d_qoy#21] +Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] (33) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt index 8105f67555c09..fee4d9431c8f2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (10) HashAggregate [ca_zip,cs_sales_price] [sum,sum] Project [cs_sales_price,ca_zip] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk,ca_zip,ca_state,cs_sales_price] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk,((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA'))),(spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)] InputAdapter WholeStageCodegen (3) Sort [cs_bill_customer_sk] @@ -35,7 +35,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (8) - Project [c_customer_sk,ca_state,ca_zip] + Project [c_customer_sk,ca_zip,ca_state] SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (5) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt index fabc11130f172..dde89e76ce48e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -TakeOrderedAndProject (22) -+- * HashAggregate (21) - +- Exchange (20) - +- * HashAggregate (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (15) - : +- * BroadcastHashJoin Inner BuildRight (14) +TakeOrderedAndProject (23) ++- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (16) + : +- * BroadcastHashJoin Inner BuildRight (15) : :- * Project (9) : : +- * BroadcastHashJoin Inner BuildRight (8) : : :- * Filter (3) @@ -16,11 +16,12 @@ TakeOrderedAndProject (22) : : +- * Filter (6) : : +- * ColumnarToRow (5) : : +- Scan parquet default.customer (4) - : +- BroadcastExchange (13) - : +- * Filter (12) - : +- * ColumnarToRow (11) - : +- Scan parquet default.customer_address (10) - +- ReusedExchange (16) + : +- BroadcastExchange (14) + : +- * Project (13) + : +- * Filter (12) + : +- * ColumnarToRow (11) + : +- Scan parquet default.customer_address (10) + +- ReusedExchange (17) (1) Scan parquet default.catalog_sales @@ -62,100 +63,104 @@ Right keys [1]: [c_customer_sk#5] Join condition: None (9) Project [codegen id : 4] -Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] +Output [4]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, (cs_sales_price#2 > 500.00) AS (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#8] Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] (10) Scan parquet default.customer_address -Output [3]: [ca_address_sk#8, ca_state#9, ca_zip#10] +Output [3]: [ca_address_sk#9, ca_state#10, ca_zip#11] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (11) ColumnarToRow [codegen id : 2] -Input [3]: [ca_address_sk#8, ca_state#9, ca_zip#10] +Input [3]: [ca_address_sk#9, ca_state#10, ca_zip#11] (12) Filter [codegen id : 2] -Input [3]: [ca_address_sk#8, ca_state#9, ca_zip#10] -Condition : isnotnull(ca_address_sk#8) +Input [3]: [ca_address_sk#9, ca_state#10, ca_zip#11] +Condition : isnotnull(ca_address_sk#9) -(13) BroadcastExchange -Input [3]: [ca_address_sk#8, ca_state#9, ca_zip#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +(13) Project [codegen id : 2] +Output [3]: [ca_address_sk#9, ca_zip#11, (substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) AS ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#12] +Input [3]: [ca_address_sk#9, ca_state#10, ca_zip#11] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastExchange +Input [3]: [ca_address_sk#9, ca_zip#11, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] + +(15) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#6] -Right keys [1]: [ca_address_sk#8] -Join condition: ((substr(ca_zip#10, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#9 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) +Right keys [1]: [ca_address_sk#9] +Join condition: (((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#12 OR (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#8) -(15) Project [codegen id : 4] -Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#10] -Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#8, ca_state#9, ca_zip#10] +(16) Project [codegen id : 4] +Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] +Input [7]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#8, ca_address_sk#9, ca_zip#11, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#12] -(16) ReusedExchange [Reuses operator id: 27] -Output [1]: [d_date_sk#12] +(17) ReusedExchange [Reuses operator id: 28] +Output [1]: [d_date_sk#14] -(17) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#14] Join condition: None -(18) Project [codegen id : 4] -Output [2]: [cs_sales_price#2, ca_zip#10] -Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#10, d_date_sk#12] +(19) Project [codegen id : 4] +Output [2]: [cs_sales_price#2, ca_zip#11] +Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#14] -(19) HashAggregate [codegen id : 4] -Input [2]: [cs_sales_price#2, ca_zip#10] -Keys [1]: [ca_zip#10] +(20) HashAggregate [codegen id : 4] +Input [2]: [cs_sales_price#2, ca_zip#11] +Keys [1]: [ca_zip#11] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [ca_zip#10, sum#14] +Aggregate Attributes [1]: [sum#15] +Results [2]: [ca_zip#11, sum#16] -(20) Exchange -Input [2]: [ca_zip#10, sum#14] -Arguments: hashpartitioning(ca_zip#10, 5), ENSURE_REQUIREMENTS, [id=#15] +(21) Exchange +Input [2]: [ca_zip#11, sum#16] +Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, [id=#17] -(21) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#10, sum#14] -Keys [1]: [ca_zip#10] +(22) HashAggregate [codegen id : 5] +Input [2]: [ca_zip#11, sum#16] +Keys [1]: [ca_zip#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#16] -Results [2]: [ca_zip#10, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#16,17,2) AS sum(cs_sales_price)#17] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#18] +Results [2]: [ca_zip#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#18,17,2) AS sum(cs_sales_price)#19] -(22) TakeOrderedAndProject -Input [2]: [ca_zip#10, sum(cs_sales_price)#17] -Arguments: 100, [ca_zip#10 ASC NULLS FIRST], [ca_zip#10, sum(cs_sales_price)#17] +(23) TakeOrderedAndProject +Input [2]: [ca_zip#11, sum(cs_sales_price)#19] +Arguments: 100, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * Project (26) - +- * Filter (25) - +- * ColumnarToRow (24) - +- Scan parquet default.date_dim (23) +BroadcastExchange (28) ++- * Project (27) + +- * Filter (26) + +- * ColumnarToRow (25) + +- Scan parquet default.date_dim (24) -(23) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#18, d_qoy#19] +(24) Scan parquet default.date_dim +Output [3]: [d_date_sk#14, d_year#20, d_qoy#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#12, d_year#18, d_qoy#19] +(25) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#14, d_year#20, d_qoy#21] -(25) Filter [codegen id : 1] -Input [3]: [d_date_sk#12, d_year#18, d_qoy#19] -Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#12)) +(26) Filter [codegen id : 1] +Input [3]: [d_date_sk#14, d_year#20, d_qoy#21] +Condition : ((((isnotnull(d_qoy#21) AND isnotnull(d_year#20)) AND (d_qoy#21 = 2)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#14)) -(26) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#18, d_qoy#19] +(27) Project [codegen id : 1] +Output [1]: [d_date_sk#14] +Input [3]: [d_date_sk#14, d_year#20, d_qoy#21] -(27) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +(28) BroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt index 0122b48fa2ecb..b9ea4ccc843a5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] Project [cs_sales_price,ca_zip] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sales_price,cs_sold_date_sk,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk,((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA'))),(spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)] Project [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] Filter [cs_bill_customer_sk] @@ -33,9 +33,10 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - Filter [ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip] + Project [ca_address_sk,ca_zip,ca_state] + Filter [ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt index c31bb7470648f..3fa91879b4632 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt @@ -182,21 +182,21 @@ Right keys [1]: [d_week_seq#69] Join condition: None (32) Project [codegen id : 11] -Output [8]: [d_week_seq#53 AS d_week_seq2#72, sun_sales#35 AS sun_sales2#73, mon_sales#36 AS mon_sales2#74, tue_sales#37 AS tue_sales2#75, wed_sales#38 AS wed_sales2#76, thu_sales#39 AS thu_sales2#77, fri_sales#40 AS fri_sales2#78, sat_sales#41 AS sat_sales2#79] +Output [8]: [sun_sales#35 AS sun_sales2#72, mon_sales#36 AS mon_sales2#73, tue_sales#37 AS tue_sales2#74, wed_sales#38 AS wed_sales2#75, thu_sales#39 AS thu_sales2#76, fri_sales#40 AS fri_sales2#77, sat_sales#41 AS sat_sales2#78, (d_week_seq#53 - 53) AS (z.d_week_seq2 - 53)#79] Input [9]: [d_week_seq#53, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#69] (33) BroadcastExchange -Input [8]: [d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#80] +Input [8]: [sun_sales2#72, mon_sales2#73, tue_sales2#74, wed_sales2#75, thu_sales2#76, fri_sales2#77, sat_sales2#78, (z.d_week_seq2 - 53)#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[7, int, true] as bigint)),false), [id=#80] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(d_week_seq2#72 - 53)] +Right keys [1]: [(z.d_week_seq2 - 53)#79] Join condition: None (35) Project [codegen id : 12] -Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#73)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#81, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#74)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#82, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#75)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#83, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#76)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#84, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#77)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#85, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#78)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#86, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#79)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#87] -Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] +Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#72)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#81, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#73)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#82, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#74)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#83, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#75)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#84, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#76)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#85, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#77)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#86, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#78)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#87] +Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, sun_sales2#72, mon_sales2#73, tue_sales2#74, wed_sales2#75, thu_sales2#76, fri_sales2#77, sat_sales2#78, (z.d_week_seq2 - 53)#79] (36) Exchange Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#81, round((mon_sales1 / mon_sales2), 2)#82, round((tue_sales1 / tue_sales2), 2)#83, round((wed_sales1 / wed_sales2), 2)#84, round((thu_sales1 / thu_sales2), 2)#85, round((fri_sales1 / fri_sales2), 2)#86, round((sat_sales1 / sat_sales2), 2)#87] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt index 036f9e8f9caa5..1e81bb28cb047 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt @@ -4,7 +4,7 @@ WholeStageCodegen (13) Exchange [d_week_seq1] #1 WholeStageCodegen (12) Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [d_week_seq1,d_week_seq2] + BroadcastHashJoin [d_week_seq1,(z.d_week_seq2 - 53)] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] @@ -44,7 +44,7 @@ WholeStageCodegen (13) InputAdapter BroadcastExchange #5 WholeStageCodegen (11) - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + Project [sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt index c31bb7470648f..3fa91879b4632 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt @@ -182,21 +182,21 @@ Right keys [1]: [d_week_seq#69] Join condition: None (32) Project [codegen id : 11] -Output [8]: [d_week_seq#53 AS d_week_seq2#72, sun_sales#35 AS sun_sales2#73, mon_sales#36 AS mon_sales2#74, tue_sales#37 AS tue_sales2#75, wed_sales#38 AS wed_sales2#76, thu_sales#39 AS thu_sales2#77, fri_sales#40 AS fri_sales2#78, sat_sales#41 AS sat_sales2#79] +Output [8]: [sun_sales#35 AS sun_sales2#72, mon_sales#36 AS mon_sales2#73, tue_sales#37 AS tue_sales2#74, wed_sales#38 AS wed_sales2#75, thu_sales#39 AS thu_sales2#76, fri_sales#40 AS fri_sales2#77, sat_sales#41 AS sat_sales2#78, (d_week_seq#53 - 53) AS (z.d_week_seq2 - 53)#79] Input [9]: [d_week_seq#53, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#69] (33) BroadcastExchange -Input [8]: [d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#80] +Input [8]: [sun_sales2#72, mon_sales2#73, tue_sales2#74, wed_sales2#75, thu_sales2#76, fri_sales2#77, sat_sales2#78, (z.d_week_seq2 - 53)#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[7, int, true] as bigint)),false), [id=#80] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(d_week_seq2#72 - 53)] +Right keys [1]: [(z.d_week_seq2 - 53)#79] Join condition: None (35) Project [codegen id : 12] -Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#73)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#81, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#74)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#82, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#75)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#83, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#76)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#84, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#77)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#85, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#78)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#86, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#79)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#87] -Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] +Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#72)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#81, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#73)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#82, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#74)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#83, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#75)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#84, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#76)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#85, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#77)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#86, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#78)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#87] +Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, sun_sales2#72, mon_sales2#73, tue_sales2#74, wed_sales2#75, thu_sales2#76, fri_sales2#77, sat_sales2#78, (z.d_week_seq2 - 53)#79] (36) Exchange Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#81, round((mon_sales1 / mon_sales2), 2)#82, round((tue_sales1 / tue_sales2), 2)#83, round((wed_sales1 / wed_sales2), 2)#84, round((thu_sales1 / thu_sales2), 2)#85, round((fri_sales1 / fri_sales2), 2)#86, round((sat_sales1 / sat_sales2), 2)#87] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index 036f9e8f9caa5..1e81bb28cb047 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -4,7 +4,7 @@ WholeStageCodegen (13) Exchange [d_week_seq1] #1 WholeStageCodegen (12) Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [d_week_seq1,d_week_seq2] + BroadcastHashJoin [d_week_seq1,(z.d_week_seq2 - 53)] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] @@ -44,7 +44,7 @@ WholeStageCodegen (13) InputAdapter BroadcastExchange #5 WholeStageCodegen (11) - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + Project [sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt index b57eca2578e00..f1b439b6931cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt @@ -219,73 +219,73 @@ Right keys [1]: [ca_zip#32] Join condition: None (39) Project [codegen id : 11] -Output [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, ca_country#33] +Output [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, upper(ca_country#33) AS upper(spark_catalog.default.customer_address.ca_country)#34] Input [7]: [s_store_sk#25, s_store_name#26, s_state#28, s_zip#29, ca_state#31, ca_zip#32, ca_country#33] (40) BroadcastExchange -Input [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, ca_country#33] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#34] +Input [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, upper(spark_catalog.default.customer_address.ca_country)#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[4, string, true]),false), [id=#35] (41) BroadcastHashJoin [codegen id : 12] Left keys [2]: [ss_store_sk#3, c_birth_country#18] -Right keys [2]: [s_store_sk#25, upper(ca_country#33)] +Right keys [2]: [s_store_sk#25, upper(spark_catalog.default.customer_address.ca_country)#34] Join condition: None (42) Project [codegen id : 12] Output [11]: [ss_net_paid#5, s_store_name#26, s_state#28, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#31] -Input [15]: [ss_store_sk#3, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, ca_country#33] +Input [15]: [ss_store_sk#3, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, upper(spark_catalog.default.customer_address.ca_country)#34] (43) HashAggregate [codegen id : 12] Input [11]: [ss_net_paid#5, s_store_name#26, s_state#28, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#31] Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#35] -Results [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Aggregate Attributes [1]: [sum#36] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#37] (44) Exchange -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#37] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#38] (45) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#37] Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] -Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] (46) HashAggregate [codegen id : 13] -Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, netpaid#39] +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, netpaid#40] Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#26] -Functions [1]: [partial_sum(netpaid#39)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43] +Functions [1]: [partial_sum(netpaid#40)] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#43, isEmpty#44] (47) Exchange -Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#43, isEmpty#44] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, 5), ENSURE_REQUIREMENTS, [id=#45] (48) HashAggregate [codegen id : 14] -Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#43, isEmpty#44] Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#26] -Functions [1]: [sum(netpaid#39)] -Aggregate Attributes [1]: [sum(netpaid#39)#45] -Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, sum(netpaid#39)#45 AS paid#46] +Functions [1]: [sum(netpaid#40)] +Aggregate Attributes [1]: [sum(netpaid#40)#46] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, sum(netpaid#40)#46 AS paid#47] (49) Filter [codegen id : 14] -Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#47] +Condition : (isnotnull(paid#47) AND (cast(paid#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (96) -+- Exchange (95) - +- * HashAggregate (94) - +- * HashAggregate (93) - +- Exchange (92) - +- * HashAggregate (91) - +- * Project (90) - +- * SortMergeJoin Inner (89) +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* HashAggregate (97) ++- Exchange (96) + +- * HashAggregate (95) + +- * HashAggregate (94) + +- Exchange (93) + +- * HashAggregate (92) + +- * Project (91) + +- * SortMergeJoin Inner (90) :- * Sort (83) : +- Exchange (82) : +- * Project (81) @@ -320,222 +320,227 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer : : +- ReusedExchange (72) : +- * Sort (79) : +- ReusedExchange (78) - +- * Sort (88) - +- Exchange (87) - +- * Filter (86) - +- * ColumnarToRow (85) - +- Scan parquet default.customer_address (84) + +- * Sort (89) + +- Exchange (88) + +- * Project (87) + +- * Filter (86) + +- * ColumnarToRow (85) + +- Scan parquet default.customer_address (84) (50) Scan parquet default.store_sales -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (52) Filter [codegen id : 2] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] -Condition : (((isnotnull(ss_ticket_number#52) AND isnotnull(ss_item_sk#49)) AND isnotnull(ss_store_sk#51)) AND isnotnull(ss_customer_sk#50)) +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) (53) Project [codegen id : 2] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (54) Scan parquet default.store -Output [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (56) Filter [codegen id : 1] -Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] -Condition : (((isnotnull(s_market_id#57) AND (s_market_id#57 = 8)) AND isnotnull(s_store_sk#55)) AND isnotnull(s_zip#59)) +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) (57) Project [codegen id : 1] -Output [4]: [s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] -Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (58) BroadcastExchange -Input [4]: [s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] (59) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#55] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None (60) Project [codegen id : 2] -Output [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] -Input [9]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] +Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] (61) Exchange -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] -Arguments: hashpartitioning(ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] (62) Sort [codegen id : 3] -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] -Arguments: [ss_item_sk#49 ASC NULLS FIRST], false, 0 +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 (63) Scan parquet default.item -Output [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] (65) Filter [codegen id : 4] -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Condition : isnotnull(i_item_sk#62) +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Condition : isnotnull(i_item_sk#63) (66) Exchange -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Arguments: hashpartitioning(i_item_sk#62, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(i_item_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] (67) Sort [codegen id : 5] -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Arguments: [i_item_sk#62 ASC NULLS FIRST], false, 0 +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [i_item_sk#63 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#49] -Right keys [1]: [i_item_sk#62] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#63] Join condition: None (69) Project [codegen id : 6] -Output [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Input [13]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Output [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Input [13]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] (70) Exchange -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Arguments: hashpartitioning(ss_customer_sk#50, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#70] (71) Sort [codegen id : 7] -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Arguments: [ss_customer_sk#50 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 (72) ReusedExchange [Reuses operator id: 16] -Output [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Output [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] (73) Sort [codegen id : 9] -Input [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] -Arguments: [c_customer_sk#70 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#50] -Right keys [1]: [c_customer_sk#70] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#71] Join condition: None (75) Project [codegen id : 10] -Output [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] -Input [16]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Input [16]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] (76) Exchange -Input [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] -Arguments: hashpartitioning(ss_ticket_number#52, ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(ss_ticket_number#53, ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#75] (77) Sort [codegen id : 11] -Input [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] -Arguments: [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [ss_ticket_number#53 ASC NULLS FIRST, ss_item_sk#50 ASC NULLS FIRST], false, 0 (78) ReusedExchange [Reuses operator id: 26] -Output [2]: [sr_item_sk#75, sr_ticket_number#76] +Output [2]: [sr_item_sk#76, sr_ticket_number#77] (79) Sort [codegen id : 13] -Input [2]: [sr_item_sk#75, sr_ticket_number#76] -Arguments: [sr_ticket_number#76 ASC NULLS FIRST, sr_item_sk#75 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#76, sr_ticket_number#77] +Arguments: [sr_ticket_number#77 ASC NULLS FIRST, sr_item_sk#76 ASC NULLS FIRST], false, 0 (80) SortMergeJoin [codegen id : 14] -Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] -Right keys [2]: [sr_ticket_number#76, sr_item_sk#75] +Left keys [2]: [ss_ticket_number#53, ss_item_sk#50] +Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] Join condition: None (81) Project [codegen id : 14] -Output [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] -Input [16]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, sr_item_sk#75, sr_ticket_number#76] +Output [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Input [16]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, sr_item_sk#76, sr_ticket_number#77] (82) Exchange -Input [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] -Arguments: hashpartitioning(c_birth_country#73, s_zip#59, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_birth_country#74, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#78] (83) Sort [codegen id : 15] -Input [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] -Arguments: [c_birth_country#73 ASC NULLS FIRST, s_zip#59 ASC NULLS FIRST], false, 0 +Input [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_birth_country#74 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 (84) Scan parquet default.customer_address -Output [3]: [ca_state#78, ca_zip#79, ca_country#80] +Output [3]: [ca_state#79, ca_zip#80, ca_country#81] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (85) ColumnarToRow [codegen id : 16] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Input [3]: [ca_state#79, ca_zip#80, ca_country#81] (86) Filter [codegen id : 16] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) +Input [3]: [ca_state#79, ca_zip#80, ca_country#81] +Condition : ((isnotnull(ca_country#81) AND isnotnull(ca_zip#80)) AND isnotnull(upper(ca_country#81))) -(87) Exchange -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: hashpartitioning(upper(ca_country#80), ca_zip#79, 5), ENSURE_REQUIREMENTS, [id=#81] +(87) Project [codegen id : 16] +Output [3]: [ca_state#79, ca_zip#80, upper(ca_country#81) AS upper(spark_catalog.default.customer_address.ca_country)#82] +Input [3]: [ca_state#79, ca_zip#80, ca_country#81] -(88) Sort [codegen id : 17] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: [upper(ca_country#80) ASC NULLS FIRST, ca_zip#79 ASC NULLS FIRST], false, 0 +(88) Exchange +Input [3]: [ca_state#79, ca_zip#80, upper(spark_catalog.default.customer_address.ca_country)#82] +Arguments: hashpartitioning(upper(spark_catalog.default.customer_address.ca_country)#82, ca_zip#80, 5), ENSURE_REQUIREMENTS, [id=#83] -(89) SortMergeJoin [codegen id : 18] -Left keys [2]: [c_birth_country#73, s_zip#59] -Right keys [2]: [upper(ca_country#80), ca_zip#79] -Join condition: None +(89) Sort [codegen id : 17] +Input [3]: [ca_state#79, ca_zip#80, upper(spark_catalog.default.customer_address.ca_country)#82] +Arguments: [upper(spark_catalog.default.customer_address.ca_country)#82 ASC NULLS FIRST, ca_zip#80 ASC NULLS FIRST], false, 0 -(90) Project [codegen id : 18] -Output [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#78] -Input [15]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, ca_state#78, ca_zip#79, ca_country#80] +(90) SortMergeJoin [codegen id : 18] +Left keys [2]: [c_birth_country#74, s_zip#60] +Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#82, ca_zip#80] +Join condition: None -(91) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#78] -Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#82] -Results [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] +(91) Project [codegen id : 18] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#79] +Input [15]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, ca_state#79, ca_zip#80, upper(spark_catalog.default.customer_address.ca_country)#82] -(92) Exchange -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] -Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#84] +(92) HashAggregate [codegen id : 18] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#79] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum#84] +Results [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#85] -(93) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] -Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#39] +(93) Exchange +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#85] +Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#86] (94) HashAggregate [codegen id : 19] -Input [1]: [netpaid#39] +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#85] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#87] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#87,17,2) AS netpaid#40] + +(95) HashAggregate [codegen id : 19] +Input [1]: [netpaid#40] Keys: [] -Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#86, count#87] -Results [2]: [sum#88, count#89] +Functions [1]: [partial_avg(netpaid#40)] +Aggregate Attributes [2]: [sum#88, count#89] +Results [2]: [sum#90, count#91] -(95) Exchange -Input [2]: [sum#88, count#89] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] +(96) Exchange +Input [2]: [sum#90, count#91] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#92] -(96) HashAggregate [codegen id : 20] -Input [2]: [sum#88, count#89] +(97) HashAggregate [codegen id : 20] +Input [2]: [sum#90, count#91] Keys: [] -Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#91] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] +Functions [1]: [avg(netpaid#40)] +Aggregate Attributes [1]: [avg(netpaid#40)#93] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#93)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#94] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt index 11f1b6b71dc01..5944b47e7a15d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt @@ -13,7 +13,7 @@ WholeStageCodegen (14) WholeStageCodegen (18) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] + SortMergeJoin [c_birth_country,s_zip,upper(spark_catalog.default.customer_address.ca_country),ca_zip] InputAdapter WholeStageCodegen (15) Sort [c_birth_country,s_zip] @@ -81,14 +81,15 @@ WholeStageCodegen (14) ReusedExchange [sr_item_sk,sr_ticket_number] #7 InputAdapter WholeStageCodegen (17) - Sort [ca_country,ca_zip] + Sort [upper(spark_catalog.default.customer_address.ca_country),ca_zip] InputAdapter - Exchange [ca_country,ca_zip] #18 + Exchange [upper(spark_catalog.default.customer_address.ca_country),ca_zip] #18 WholeStageCodegen (16) - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + Project [ca_state,ca_zip,ca_country] + Filter [ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name,s_store_name] #1 @@ -100,7 +101,7 @@ WholeStageCodegen (14) WholeStageCodegen (12) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [ss_store_sk,c_birth_country,s_store_sk,ca_country] + BroadcastHashJoin [ss_store_sk,c_birth_country,s_store_sk,upper(spark_catalog.default.customer_address.ca_country)] Project [ss_store_sk,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt index c8d73ab27c82b..41b4a970960c7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -* Filter (46) -+- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) +* Filter (47) ++- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (33) : +- * BroadcastHashJoin Inner BuildRight (32) : :- * Project (27) @@ -41,10 +41,11 @@ : +- * Filter (30) : +- * ColumnarToRow (29) : +- Scan parquet default.customer (28) - +- BroadcastExchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.customer_address (34) + +- BroadcastExchange (38) + +- * Project (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.customer_address (34) (1) Scan parquet default.store_sales @@ -207,211 +208,238 @@ Input [3]: [ca_state#30, ca_zip#31, ca_country#32] Input [3]: [ca_state#30, ca_zip#31, ca_country#32] Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) -(37) BroadcastExchange +(37) Project [codegen id : 8] +Output [3]: [ca_state#30, ca_zip#31, upper(ca_country#32) AS upper(spark_catalog.default.customer_address.ca_country)#33] Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#33] -(38) BroadcastHashJoin [codegen id : 9] +(38) BroadcastExchange +Input [3]: [ca_state#30, ca_zip#31, upper(spark_catalog.default.customer_address.ca_country)#33] +Arguments: HashedRelationBroadcastMode(List(input[2, string, true], input[1, string, true]),false), [id=#34] + +(39) BroadcastHashJoin [codegen id : 9] Left keys [2]: [c_birth_country#28, s_zip#16] -Right keys [2]: [upper(ca_country#32), ca_zip#31] +Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#33, ca_zip#31] Join condition: None -(39) Project [codegen id : 9] +(40) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, ca_state#30] -Input [15]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, c_birth_country#28, ca_state#30, ca_zip#31, ca_country#32] +Input [15]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, c_birth_country#28, ca_state#30, ca_zip#31, upper(spark_catalog.default.customer_address.ca_country)#33] -(40) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, ca_state#30] Keys [10]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#34] -Results [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#35] +Aggregate Attributes [1]: [sum#35] +Results [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#36] -(41) Exchange -Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#35] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#36] +(42) Exchange +Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#36] +Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#37] -(42) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#35] +(43) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#36] Keys [10]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#37] -Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#37,17,2) AS netpaid#38] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] +Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] -(43) HashAggregate [codegen id : 10] -Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, netpaid#38] +(44) HashAggregate [codegen id : 10] +Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, netpaid#39] Keys [3]: [c_last_name#27, c_first_name#26, s_store_name#13] -Functions [1]: [partial_sum(netpaid#38)] -Aggregate Attributes [2]: [sum#39, isEmpty#40] -Results [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42] +Functions [1]: [partial_sum(netpaid#39)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#42, isEmpty#43] -(44) Exchange -Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#43] +(45) Exchange +Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#42, isEmpty#43] +Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#44] -(45) HashAggregate [codegen id : 11] -Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42] +(46) HashAggregate [codegen id : 11] +Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#42, isEmpty#43] Keys [3]: [c_last_name#27, c_first_name#26, s_store_name#13] -Functions [1]: [sum(netpaid#38)] -Aggregate Attributes [1]: [sum(netpaid#38)#44] -Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, sum(netpaid#38)#44 AS paid#45] +Functions [1]: [sum(netpaid#39)] +Aggregate Attributes [1]: [sum(netpaid#39)#45] +Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, sum(netpaid#39)#45 AS paid#46] -(46) Filter [codegen id : 11] -Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#45] -Condition : (isnotnull(paid#45) AND (cast(paid#45 as decimal(33,8)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(33,8)))) +(47) Filter [codegen id : 11] +Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#46] +Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#46, [id=#47] -* HashAggregate (73) -+- Exchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- Exchange (69) - +- * HashAggregate (68) - +- * Project (67) - +- * BroadcastHashJoin Inner BuildRight (66) - :- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Project (52) - : : : : +- * SortMergeJoin Inner (51) - : : : : :- * Sort (48) - : : : : : +- ReusedExchange (47) - : : : : +- * Sort (50) - : : : : +- ReusedExchange (49) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * Filter (58) - : : +- * ColumnarToRow (57) - : : +- Scan parquet default.item (56) - : +- ReusedExchange (62) - +- ReusedExchange (65) - - -(47) ReusedExchange [Reuses operator id: 5] -Output [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] - -(48) Sort [codegen id : 2] -Input [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] -Arguments: [ss_ticket_number#51 ASC NULLS FIRST, ss_item_sk#48 ASC NULLS FIRST], false, 0 - -(49) ReusedExchange [Reuses operator id: 11] -Output [2]: [sr_item_sk#53, sr_ticket_number#54] - -(50) Sort [codegen id : 4] -Input [2]: [sr_item_sk#53, sr_ticket_number#54] -Arguments: [sr_ticket_number#54 ASC NULLS FIRST, sr_item_sk#53 ASC NULLS FIRST], false, 0 - -(51) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#51, ss_item_sk#48] -Right keys [2]: [sr_ticket_number#54, sr_item_sk#53] +Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] +* HashAggregate (78) ++- Exchange (77) + +- * HashAggregate (76) + +- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Project (56) + : : : +- * BroadcastHashJoin Inner BuildRight (55) + : : : :- * Project (53) + : : : : +- * SortMergeJoin Inner (52) + : : : : :- * Sort (49) + : : : : : +- ReusedExchange (48) + : : : : +- * Sort (51) + : : : : +- ReusedExchange (50) + : : : +- ReusedExchange (54) + : : +- BroadcastExchange (60) + : : +- * Filter (59) + : : +- * ColumnarToRow (58) + : : +- Scan parquet default.item (57) + : +- ReusedExchange (63) + +- BroadcastExchange (70) + +- * Project (69) + +- * Filter (68) + +- * ColumnarToRow (67) + +- Scan parquet default.customer_address (66) + + +(48) ReusedExchange [Reuses operator id: 5] +Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] + +(49) Sort [codegen id : 2] +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST], false, 0 + +(50) ReusedExchange [Reuses operator id: 11] +Output [2]: [sr_item_sk#54, sr_ticket_number#55] + +(51) Sort [codegen id : 4] +Input [2]: [sr_item_sk#54, sr_ticket_number#55] +Arguments: [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST], false, 0 + +(52) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] +Right keys [2]: [sr_ticket_number#55, sr_item_sk#54] Join condition: None -(52) Project [codegen id : 9] -Output [4]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52] -Input [7]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, sr_item_sk#53, sr_ticket_number#54] +(53) Project [codegen id : 9] +Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -(53) ReusedExchange [Reuses operator id: 19] -Output [4]: [s_store_sk#55, s_store_name#56, s_state#57, s_zip#58] +(54) ReusedExchange [Reuses operator id: 19] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -(54) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#50] -Right keys [1]: [s_store_sk#55] +(55) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#51] +Right keys [1]: [s_store_sk#56] Join condition: None -(55) Project [codegen id : 9] -Output [6]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58] -Input [8]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52, s_store_sk#55, s_store_name#56, s_state#57, s_zip#58] +(56) Project [codegen id : 9] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] +Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -(56) Scan parquet default.item -Output [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +(57) Scan parquet default.item +Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(57) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +(58) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -(58) Filter [codegen id : 6] -Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -Condition : isnotnull(i_item_sk#59) +(59) Filter [codegen id : 6] +Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] +Condition : isnotnull(i_item_sk#60) -(59) BroadcastExchange -Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] +(60) BroadcastExchange +Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] -(60) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#48] -Right keys [1]: [i_item_sk#59] +(61) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#49] +Right keys [1]: [i_item_sk#60] Join condition: None -(61) Project [codegen id : 9] -Output [10]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -Input [12]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +(62) Project [codegen id : 9] +Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -(62) ReusedExchange [Reuses operator id: 31] -Output [4]: [c_customer_sk#66, c_first_name#67, c_last_name#68, c_birth_country#69] +(63) ReusedExchange [Reuses operator id: 31] +Output [4]: [c_customer_sk#67, c_first_name#68, c_last_name#69, c_birth_country#70] -(63) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#49] -Right keys [1]: [c_customer_sk#66] +(64) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#50] +Right keys [1]: [c_customer_sk#67] Join condition: None -(64) Project [codegen id : 9] -Output [12]: [ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, c_birth_country#69] -Input [14]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_customer_sk#66, c_first_name#67, c_last_name#68, c_birth_country#69] +(65) Project [codegen id : 9] +Output [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, c_birth_country#70] +Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_customer_sk#67, c_first_name#68, c_last_name#69, c_birth_country#70] + +(66) Scan parquet default.customer_address +Output [3]: [ca_state#71, ca_zip#72, ca_country#73] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] +ReadSchema: struct + +(67) ColumnarToRow [codegen id : 8] +Input [3]: [ca_state#71, ca_zip#72, ca_country#73] + +(68) Filter [codegen id : 8] +Input [3]: [ca_state#71, ca_zip#72, ca_country#73] +Condition : ((isnotnull(ca_country#73) AND isnotnull(ca_zip#72)) AND isnotnull(upper(ca_country#73))) + +(69) Project [codegen id : 8] +Output [3]: [ca_state#71, ca_zip#72, upper(ca_country#73) AS upper(spark_catalog.default.customer_address.ca_country)#74] +Input [3]: [ca_state#71, ca_zip#72, ca_country#73] -(65) ReusedExchange [Reuses operator id: 37] -Output [3]: [ca_state#70, ca_zip#71, ca_country#72] +(70) BroadcastExchange +Input [3]: [ca_state#71, ca_zip#72, upper(spark_catalog.default.customer_address.ca_country)#74] +Arguments: HashedRelationBroadcastMode(List(input[2, string, true], input[1, string, true]),false), [id=#75] -(66) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#69, s_zip#58] -Right keys [2]: [upper(ca_country#72), ca_zip#71] +(71) BroadcastHashJoin [codegen id : 9] +Left keys [2]: [c_birth_country#70, s_zip#59] +Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#74, ca_zip#72] Join condition: None -(67) Project [codegen id : 9] -Output [11]: [ss_net_paid#52, s_store_name#56, s_state#57, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, ca_state#70] -Input [15]: [ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, c_birth_country#69, ca_state#70, ca_zip#71, ca_country#72] - -(68) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#52, s_store_name#56, s_state#57, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, ca_state#70] -Keys [10]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [sum#73] -Results [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] - -(69) Exchange -Input [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] -Arguments: hashpartitioning(c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, 5), ENSURE_REQUIREMENTS, [id=#75] - -(70) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] -Keys [10]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61] -Functions [1]: [sum(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#52))#76] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#52))#76,17,2) AS netpaid#38] - -(71) HashAggregate [codegen id : 10] -Input [1]: [netpaid#38] +(72) Project [codegen id : 9] +Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, ca_state#71] +Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, c_birth_country#70, ca_state#71, ca_zip#72, upper(spark_catalog.default.customer_address.ca_country)#74] + +(73) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, ca_state#71] +Keys [10]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum#76] +Results [11]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, sum#77] + +(74) Exchange +Input [11]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, sum#77] +Arguments: hashpartitioning(c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, 5), ENSURE_REQUIREMENTS, [id=#78] + +(75) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, sum#77] +Keys [10]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62] +Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#79] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#79,17,2) AS netpaid#39] + +(76) HashAggregate [codegen id : 10] +Input [1]: [netpaid#39] Keys: [] -Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#77, count#78] -Results [2]: [sum#79, count#80] +Functions [1]: [partial_avg(netpaid#39)] +Aggregate Attributes [2]: [sum#80, count#81] +Results [2]: [sum#82, count#83] -(72) Exchange -Input [2]: [sum#79, count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +(77) Exchange +Input [2]: [sum#82, count#83] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#84] -(73) HashAggregate [codegen id : 11] -Input [2]: [sum#79, count#80] +(78) HashAggregate [codegen id : 11] +Input [2]: [sum#82, count#83] Keys: [] -Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#82] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#82)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#83] +Functions [1]: [avg(netpaid#39)] +Aggregate Attributes [1]: [avg(netpaid#39)#85] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#85)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#86] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index 2d9ef020540b8..f60055c66c5cb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -13,7 +13,7 @@ WholeStageCodegen (11) WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + BroadcastHashJoin [c_birth_country,s_zip,upper(spark_catalog.default.customer_address.ca_country),ca_zip] Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -44,7 +44,13 @@ WholeStageCodegen (11) InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 + BroadcastExchange #12 + WholeStageCodegen (8) + Project [ca_state,ca_zip,ca_country] + Filter [ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name,s_store_name] #1 @@ -56,7 +62,7 @@ WholeStageCodegen (11) WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + BroadcastHashJoin [c_birth_country,s_zip,upper(spark_catalog.default.customer_address.ca_country),ca_zip] Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -112,7 +118,8 @@ WholeStageCodegen (11) InputAdapter BroadcastExchange #8 WholeStageCodegen (8) - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + Project [ca_state,ca_zip,ca_country] + Filter [ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt index 1814c7de8204d..68170dd60175e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt @@ -219,73 +219,73 @@ Right keys [1]: [ca_zip#32] Join condition: None (39) Project [codegen id : 11] -Output [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, ca_country#33] +Output [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, upper(ca_country#33) AS upper(spark_catalog.default.customer_address.ca_country)#34] Input [7]: [s_store_sk#25, s_store_name#26, s_state#28, s_zip#29, ca_state#31, ca_zip#32, ca_country#33] (40) BroadcastExchange -Input [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, ca_country#33] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#34] +Input [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, upper(spark_catalog.default.customer_address.ca_country)#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[4, string, true]),false), [id=#35] (41) BroadcastHashJoin [codegen id : 12] Left keys [2]: [ss_store_sk#3, c_birth_country#18] -Right keys [2]: [s_store_sk#25, upper(ca_country#33)] +Right keys [2]: [s_store_sk#25, upper(spark_catalog.default.customer_address.ca_country)#34] Join condition: None (42) Project [codegen id : 12] Output [11]: [ss_net_paid#5, s_store_name#26, s_state#28, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#31] -Input [15]: [ss_store_sk#3, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, ca_country#33] +Input [15]: [ss_store_sk#3, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, upper(spark_catalog.default.customer_address.ca_country)#34] (43) HashAggregate [codegen id : 12] Input [11]: [ss_net_paid#5, s_store_name#26, s_state#28, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#31] Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#35] -Results [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Aggregate Attributes [1]: [sum#36] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#37] (44) Exchange -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#37] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#38] (45) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#37] Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] -Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] (46) HashAggregate [codegen id : 13] -Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, netpaid#39] +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, netpaid#40] Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#26] -Functions [1]: [partial_sum(netpaid#39)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43] +Functions [1]: [partial_sum(netpaid#40)] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#43, isEmpty#44] (47) Exchange -Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#43, isEmpty#44] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, 5), ENSURE_REQUIREMENTS, [id=#45] (48) HashAggregate [codegen id : 14] -Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#43, isEmpty#44] Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#26] -Functions [1]: [sum(netpaid#39)] -Aggregate Attributes [1]: [sum(netpaid#39)#45] -Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, sum(netpaid#39)#45 AS paid#46] +Functions [1]: [sum(netpaid#40)] +Aggregate Attributes [1]: [sum(netpaid#40)#46] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, sum(netpaid#40)#46 AS paid#47] (49) Filter [codegen id : 14] -Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#47] +Condition : (isnotnull(paid#47) AND (cast(paid#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (96) -+- Exchange (95) - +- * HashAggregate (94) - +- * HashAggregate (93) - +- Exchange (92) - +- * HashAggregate (91) - +- * Project (90) - +- * SortMergeJoin Inner (89) +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* HashAggregate (97) ++- Exchange (96) + +- * HashAggregate (95) + +- * HashAggregate (94) + +- Exchange (93) + +- * HashAggregate (92) + +- * Project (91) + +- * SortMergeJoin Inner (90) :- * Sort (83) : +- Exchange (82) : +- * Project (81) @@ -320,222 +320,227 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer : : +- ReusedExchange (72) : +- * Sort (79) : +- ReusedExchange (78) - +- * Sort (88) - +- Exchange (87) - +- * Filter (86) - +- * ColumnarToRow (85) - +- Scan parquet default.customer_address (84) + +- * Sort (89) + +- Exchange (88) + +- * Project (87) + +- * Filter (86) + +- * ColumnarToRow (85) + +- Scan parquet default.customer_address (84) (50) Scan parquet default.store_sales -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (52) Filter [codegen id : 2] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] -Condition : (((isnotnull(ss_ticket_number#52) AND isnotnull(ss_item_sk#49)) AND isnotnull(ss_store_sk#51)) AND isnotnull(ss_customer_sk#50)) +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) (53) Project [codegen id : 2] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (54) Scan parquet default.store -Output [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (56) Filter [codegen id : 1] -Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] -Condition : (((isnotnull(s_market_id#57) AND (s_market_id#57 = 8)) AND isnotnull(s_store_sk#55)) AND isnotnull(s_zip#59)) +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) (57) Project [codegen id : 1] -Output [4]: [s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] -Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (58) BroadcastExchange -Input [4]: [s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] (59) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#55] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None (60) Project [codegen id : 2] -Output [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] -Input [9]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] +Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] (61) Exchange -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] -Arguments: hashpartitioning(ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] (62) Sort [codegen id : 3] -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] -Arguments: [ss_item_sk#49 ASC NULLS FIRST], false, 0 +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 (63) Scan parquet default.item -Output [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] (65) Filter [codegen id : 4] -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Condition : isnotnull(i_item_sk#62) +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Condition : isnotnull(i_item_sk#63) (66) Exchange -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Arguments: hashpartitioning(i_item_sk#62, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(i_item_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] (67) Sort [codegen id : 5] -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Arguments: [i_item_sk#62 ASC NULLS FIRST], false, 0 +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [i_item_sk#63 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#49] -Right keys [1]: [i_item_sk#62] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#63] Join condition: None (69) Project [codegen id : 6] -Output [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Input [13]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Output [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Input [13]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] (70) Exchange -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Arguments: hashpartitioning(ss_customer_sk#50, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#70] (71) Sort [codegen id : 7] -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Arguments: [ss_customer_sk#50 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 (72) ReusedExchange [Reuses operator id: 16] -Output [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Output [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] (73) Sort [codegen id : 9] -Input [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] -Arguments: [c_customer_sk#70 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#50] -Right keys [1]: [c_customer_sk#70] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#71] Join condition: None (75) Project [codegen id : 10] -Output [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] -Input [16]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Input [16]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] (76) Exchange -Input [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] -Arguments: hashpartitioning(ss_ticket_number#52, ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(ss_ticket_number#53, ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#75] (77) Sort [codegen id : 11] -Input [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] -Arguments: [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [ss_ticket_number#53 ASC NULLS FIRST, ss_item_sk#50 ASC NULLS FIRST], false, 0 (78) ReusedExchange [Reuses operator id: 26] -Output [2]: [sr_item_sk#75, sr_ticket_number#76] +Output [2]: [sr_item_sk#76, sr_ticket_number#77] (79) Sort [codegen id : 13] -Input [2]: [sr_item_sk#75, sr_ticket_number#76] -Arguments: [sr_ticket_number#76 ASC NULLS FIRST, sr_item_sk#75 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#76, sr_ticket_number#77] +Arguments: [sr_ticket_number#77 ASC NULLS FIRST, sr_item_sk#76 ASC NULLS FIRST], false, 0 (80) SortMergeJoin [codegen id : 14] -Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] -Right keys [2]: [sr_ticket_number#76, sr_item_sk#75] +Left keys [2]: [ss_ticket_number#53, ss_item_sk#50] +Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] Join condition: None (81) Project [codegen id : 14] -Output [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] -Input [16]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, sr_item_sk#75, sr_ticket_number#76] +Output [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Input [16]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, sr_item_sk#76, sr_ticket_number#77] (82) Exchange -Input [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] -Arguments: hashpartitioning(c_birth_country#73, s_zip#59, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_birth_country#74, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#78] (83) Sort [codegen id : 15] -Input [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] -Arguments: [c_birth_country#73 ASC NULLS FIRST, s_zip#59 ASC NULLS FIRST], false, 0 +Input [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_birth_country#74 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 (84) Scan parquet default.customer_address -Output [3]: [ca_state#78, ca_zip#79, ca_country#80] +Output [3]: [ca_state#79, ca_zip#80, ca_country#81] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (85) ColumnarToRow [codegen id : 16] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Input [3]: [ca_state#79, ca_zip#80, ca_country#81] (86) Filter [codegen id : 16] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) +Input [3]: [ca_state#79, ca_zip#80, ca_country#81] +Condition : ((isnotnull(ca_country#81) AND isnotnull(ca_zip#80)) AND isnotnull(upper(ca_country#81))) -(87) Exchange -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: hashpartitioning(upper(ca_country#80), ca_zip#79, 5), ENSURE_REQUIREMENTS, [id=#81] +(87) Project [codegen id : 16] +Output [3]: [ca_state#79, ca_zip#80, upper(ca_country#81) AS upper(spark_catalog.default.customer_address.ca_country)#82] +Input [3]: [ca_state#79, ca_zip#80, ca_country#81] -(88) Sort [codegen id : 17] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: [upper(ca_country#80) ASC NULLS FIRST, ca_zip#79 ASC NULLS FIRST], false, 0 +(88) Exchange +Input [3]: [ca_state#79, ca_zip#80, upper(spark_catalog.default.customer_address.ca_country)#82] +Arguments: hashpartitioning(upper(spark_catalog.default.customer_address.ca_country)#82, ca_zip#80, 5), ENSURE_REQUIREMENTS, [id=#83] -(89) SortMergeJoin [codegen id : 18] -Left keys [2]: [c_birth_country#73, s_zip#59] -Right keys [2]: [upper(ca_country#80), ca_zip#79] -Join condition: None +(89) Sort [codegen id : 17] +Input [3]: [ca_state#79, ca_zip#80, upper(spark_catalog.default.customer_address.ca_country)#82] +Arguments: [upper(spark_catalog.default.customer_address.ca_country)#82 ASC NULLS FIRST, ca_zip#80 ASC NULLS FIRST], false, 0 -(90) Project [codegen id : 18] -Output [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#78] -Input [15]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, ca_state#78, ca_zip#79, ca_country#80] +(90) SortMergeJoin [codegen id : 18] +Left keys [2]: [c_birth_country#74, s_zip#60] +Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#82, ca_zip#80] +Join condition: None -(91) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#78] -Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#82] -Results [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] +(91) Project [codegen id : 18] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#79] +Input [15]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, ca_state#79, ca_zip#80, upper(spark_catalog.default.customer_address.ca_country)#82] -(92) Exchange -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] -Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#84] +(92) HashAggregate [codegen id : 18] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#79] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum#84] +Results [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#85] -(93) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] -Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#39] +(93) Exchange +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#85] +Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#86] (94) HashAggregate [codegen id : 19] -Input [1]: [netpaid#39] +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#85] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#87] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#87,17,2) AS netpaid#40] + +(95) HashAggregate [codegen id : 19] +Input [1]: [netpaid#40] Keys: [] -Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#86, count#87] -Results [2]: [sum#88, count#89] +Functions [1]: [partial_avg(netpaid#40)] +Aggregate Attributes [2]: [sum#88, count#89] +Results [2]: [sum#90, count#91] -(95) Exchange -Input [2]: [sum#88, count#89] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] +(96) Exchange +Input [2]: [sum#90, count#91] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#92] -(96) HashAggregate [codegen id : 20] -Input [2]: [sum#88, count#89] +(97) HashAggregate [codegen id : 20] +Input [2]: [sum#90, count#91] Keys: [] -Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#91] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] +Functions [1]: [avg(netpaid#40)] +Aggregate Attributes [1]: [avg(netpaid#40)#93] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#93)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#94] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt index 11f1b6b71dc01..5944b47e7a15d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt @@ -13,7 +13,7 @@ WholeStageCodegen (14) WholeStageCodegen (18) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] + SortMergeJoin [c_birth_country,s_zip,upper(spark_catalog.default.customer_address.ca_country),ca_zip] InputAdapter WholeStageCodegen (15) Sort [c_birth_country,s_zip] @@ -81,14 +81,15 @@ WholeStageCodegen (14) ReusedExchange [sr_item_sk,sr_ticket_number] #7 InputAdapter WholeStageCodegen (17) - Sort [ca_country,ca_zip] + Sort [upper(spark_catalog.default.customer_address.ca_country),ca_zip] InputAdapter - Exchange [ca_country,ca_zip] #18 + Exchange [upper(spark_catalog.default.customer_address.ca_country),ca_zip] #18 WholeStageCodegen (16) - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + Project [ca_state,ca_zip,ca_country] + Filter [ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name,s_store_name] #1 @@ -100,7 +101,7 @@ WholeStageCodegen (14) WholeStageCodegen (12) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [ss_store_sk,c_birth_country,s_store_sk,ca_country] + BroadcastHashJoin [ss_store_sk,c_birth_country,s_store_sk,upper(spark_catalog.default.customer_address.ca_country)] Project [ss_store_sk,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt index 443ca4d35b780..c0e873fe39fc2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -* Filter (46) -+- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) +* Filter (47) ++- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (33) : +- * BroadcastHashJoin Inner BuildRight (32) : :- * Project (27) @@ -41,10 +41,11 @@ : +- * Filter (30) : +- * ColumnarToRow (29) : +- Scan parquet default.customer (28) - +- BroadcastExchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.customer_address (34) + +- BroadcastExchange (38) + +- * Project (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.customer_address (34) (1) Scan parquet default.store_sales @@ -207,211 +208,238 @@ Input [3]: [ca_state#30, ca_zip#31, ca_country#32] Input [3]: [ca_state#30, ca_zip#31, ca_country#32] Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) -(37) BroadcastExchange +(37) Project [codegen id : 8] +Output [3]: [ca_state#30, ca_zip#31, upper(ca_country#32) AS upper(spark_catalog.default.customer_address.ca_country)#33] Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#33] -(38) BroadcastHashJoin [codegen id : 9] +(38) BroadcastExchange +Input [3]: [ca_state#30, ca_zip#31, upper(spark_catalog.default.customer_address.ca_country)#33] +Arguments: HashedRelationBroadcastMode(List(input[2, string, true], input[1, string, true]),false), [id=#34] + +(39) BroadcastHashJoin [codegen id : 9] Left keys [2]: [c_birth_country#28, s_zip#16] -Right keys [2]: [upper(ca_country#32), ca_zip#31] +Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#33, ca_zip#31] Join condition: None -(39) Project [codegen id : 9] +(40) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, ca_state#30] -Input [15]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, c_birth_country#28, ca_state#30, ca_zip#31, ca_country#32] +Input [15]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, c_birth_country#28, ca_state#30, ca_zip#31, upper(spark_catalog.default.customer_address.ca_country)#33] -(40) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, ca_state#30] Keys [10]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#34] -Results [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#35] +Aggregate Attributes [1]: [sum#35] +Results [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#36] -(41) Exchange -Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#35] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#36] +(42) Exchange +Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#36] +Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#37] -(42) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#35] +(43) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#36] Keys [10]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#37] -Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#37,17,2) AS netpaid#38] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] +Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] -(43) HashAggregate [codegen id : 10] -Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, netpaid#38] +(44) HashAggregate [codegen id : 10] +Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, netpaid#39] Keys [3]: [c_last_name#27, c_first_name#26, s_store_name#13] -Functions [1]: [partial_sum(netpaid#38)] -Aggregate Attributes [2]: [sum#39, isEmpty#40] -Results [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42] +Functions [1]: [partial_sum(netpaid#39)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#42, isEmpty#43] -(44) Exchange -Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#43] +(45) Exchange +Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#42, isEmpty#43] +Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#44] -(45) HashAggregate [codegen id : 11] -Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42] +(46) HashAggregate [codegen id : 11] +Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#42, isEmpty#43] Keys [3]: [c_last_name#27, c_first_name#26, s_store_name#13] -Functions [1]: [sum(netpaid#38)] -Aggregate Attributes [1]: [sum(netpaid#38)#44] -Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, sum(netpaid#38)#44 AS paid#45] +Functions [1]: [sum(netpaid#39)] +Aggregate Attributes [1]: [sum(netpaid#39)#45] +Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, sum(netpaid#39)#45 AS paid#46] -(46) Filter [codegen id : 11] -Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#45] -Condition : (isnotnull(paid#45) AND (cast(paid#45 as decimal(33,8)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(33,8)))) +(47) Filter [codegen id : 11] +Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#46] +Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#46, [id=#47] -* HashAggregate (73) -+- Exchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- Exchange (69) - +- * HashAggregate (68) - +- * Project (67) - +- * BroadcastHashJoin Inner BuildRight (66) - :- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Project (52) - : : : : +- * SortMergeJoin Inner (51) - : : : : :- * Sort (48) - : : : : : +- ReusedExchange (47) - : : : : +- * Sort (50) - : : : : +- ReusedExchange (49) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * Filter (58) - : : +- * ColumnarToRow (57) - : : +- Scan parquet default.item (56) - : +- ReusedExchange (62) - +- ReusedExchange (65) - - -(47) ReusedExchange [Reuses operator id: 5] -Output [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] - -(48) Sort [codegen id : 2] -Input [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] -Arguments: [ss_ticket_number#51 ASC NULLS FIRST, ss_item_sk#48 ASC NULLS FIRST], false, 0 - -(49) ReusedExchange [Reuses operator id: 11] -Output [2]: [sr_item_sk#53, sr_ticket_number#54] - -(50) Sort [codegen id : 4] -Input [2]: [sr_item_sk#53, sr_ticket_number#54] -Arguments: [sr_ticket_number#54 ASC NULLS FIRST, sr_item_sk#53 ASC NULLS FIRST], false, 0 - -(51) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#51, ss_item_sk#48] -Right keys [2]: [sr_ticket_number#54, sr_item_sk#53] +Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] +* HashAggregate (78) ++- Exchange (77) + +- * HashAggregate (76) + +- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Project (56) + : : : +- * BroadcastHashJoin Inner BuildRight (55) + : : : :- * Project (53) + : : : : +- * SortMergeJoin Inner (52) + : : : : :- * Sort (49) + : : : : : +- ReusedExchange (48) + : : : : +- * Sort (51) + : : : : +- ReusedExchange (50) + : : : +- ReusedExchange (54) + : : +- BroadcastExchange (60) + : : +- * Filter (59) + : : +- * ColumnarToRow (58) + : : +- Scan parquet default.item (57) + : +- ReusedExchange (63) + +- BroadcastExchange (70) + +- * Project (69) + +- * Filter (68) + +- * ColumnarToRow (67) + +- Scan parquet default.customer_address (66) + + +(48) ReusedExchange [Reuses operator id: 5] +Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] + +(49) Sort [codegen id : 2] +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST], false, 0 + +(50) ReusedExchange [Reuses operator id: 11] +Output [2]: [sr_item_sk#54, sr_ticket_number#55] + +(51) Sort [codegen id : 4] +Input [2]: [sr_item_sk#54, sr_ticket_number#55] +Arguments: [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST], false, 0 + +(52) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] +Right keys [2]: [sr_ticket_number#55, sr_item_sk#54] Join condition: None -(52) Project [codegen id : 9] -Output [4]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52] -Input [7]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, sr_item_sk#53, sr_ticket_number#54] +(53) Project [codegen id : 9] +Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -(53) ReusedExchange [Reuses operator id: 19] -Output [4]: [s_store_sk#55, s_store_name#56, s_state#57, s_zip#58] +(54) ReusedExchange [Reuses operator id: 19] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -(54) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#50] -Right keys [1]: [s_store_sk#55] +(55) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#51] +Right keys [1]: [s_store_sk#56] Join condition: None -(55) Project [codegen id : 9] -Output [6]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58] -Input [8]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52, s_store_sk#55, s_store_name#56, s_state#57, s_zip#58] +(56) Project [codegen id : 9] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] +Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -(56) Scan parquet default.item -Output [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +(57) Scan parquet default.item +Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(57) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +(58) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -(58) Filter [codegen id : 6] -Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -Condition : isnotnull(i_item_sk#59) +(59) Filter [codegen id : 6] +Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] +Condition : isnotnull(i_item_sk#60) -(59) BroadcastExchange -Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] +(60) BroadcastExchange +Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] -(60) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#48] -Right keys [1]: [i_item_sk#59] +(61) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#49] +Right keys [1]: [i_item_sk#60] Join condition: None -(61) Project [codegen id : 9] -Output [10]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -Input [12]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +(62) Project [codegen id : 9] +Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -(62) ReusedExchange [Reuses operator id: 31] -Output [4]: [c_customer_sk#66, c_first_name#67, c_last_name#68, c_birth_country#69] +(63) ReusedExchange [Reuses operator id: 31] +Output [4]: [c_customer_sk#67, c_first_name#68, c_last_name#69, c_birth_country#70] -(63) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#49] -Right keys [1]: [c_customer_sk#66] +(64) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#50] +Right keys [1]: [c_customer_sk#67] Join condition: None -(64) Project [codegen id : 9] -Output [12]: [ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, c_birth_country#69] -Input [14]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_customer_sk#66, c_first_name#67, c_last_name#68, c_birth_country#69] +(65) Project [codegen id : 9] +Output [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, c_birth_country#70] +Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_customer_sk#67, c_first_name#68, c_last_name#69, c_birth_country#70] + +(66) Scan parquet default.customer_address +Output [3]: [ca_state#71, ca_zip#72, ca_country#73] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] +ReadSchema: struct + +(67) ColumnarToRow [codegen id : 8] +Input [3]: [ca_state#71, ca_zip#72, ca_country#73] + +(68) Filter [codegen id : 8] +Input [3]: [ca_state#71, ca_zip#72, ca_country#73] +Condition : ((isnotnull(ca_country#73) AND isnotnull(ca_zip#72)) AND isnotnull(upper(ca_country#73))) + +(69) Project [codegen id : 8] +Output [3]: [ca_state#71, ca_zip#72, upper(ca_country#73) AS upper(spark_catalog.default.customer_address.ca_country)#74] +Input [3]: [ca_state#71, ca_zip#72, ca_country#73] -(65) ReusedExchange [Reuses operator id: 37] -Output [3]: [ca_state#70, ca_zip#71, ca_country#72] +(70) BroadcastExchange +Input [3]: [ca_state#71, ca_zip#72, upper(spark_catalog.default.customer_address.ca_country)#74] +Arguments: HashedRelationBroadcastMode(List(input[2, string, true], input[1, string, true]),false), [id=#75] -(66) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#69, s_zip#58] -Right keys [2]: [upper(ca_country#72), ca_zip#71] +(71) BroadcastHashJoin [codegen id : 9] +Left keys [2]: [c_birth_country#70, s_zip#59] +Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#74, ca_zip#72] Join condition: None -(67) Project [codegen id : 9] -Output [11]: [ss_net_paid#52, s_store_name#56, s_state#57, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, ca_state#70] -Input [15]: [ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, c_birth_country#69, ca_state#70, ca_zip#71, ca_country#72] - -(68) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#52, s_store_name#56, s_state#57, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, ca_state#70] -Keys [10]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [sum#73] -Results [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] - -(69) Exchange -Input [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] -Arguments: hashpartitioning(c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, 5), ENSURE_REQUIREMENTS, [id=#75] - -(70) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] -Keys [10]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61] -Functions [1]: [sum(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#52))#76] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#52))#76,17,2) AS netpaid#38] - -(71) HashAggregate [codegen id : 10] -Input [1]: [netpaid#38] +(72) Project [codegen id : 9] +Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, ca_state#71] +Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, c_birth_country#70, ca_state#71, ca_zip#72, upper(spark_catalog.default.customer_address.ca_country)#74] + +(73) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, ca_state#71] +Keys [10]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum#76] +Results [11]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, sum#77] + +(74) Exchange +Input [11]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, sum#77] +Arguments: hashpartitioning(c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, 5), ENSURE_REQUIREMENTS, [id=#78] + +(75) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, sum#77] +Keys [10]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62] +Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#79] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#79,17,2) AS netpaid#39] + +(76) HashAggregate [codegen id : 10] +Input [1]: [netpaid#39] Keys: [] -Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#77, count#78] -Results [2]: [sum#79, count#80] +Functions [1]: [partial_avg(netpaid#39)] +Aggregate Attributes [2]: [sum#80, count#81] +Results [2]: [sum#82, count#83] -(72) Exchange -Input [2]: [sum#79, count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +(77) Exchange +Input [2]: [sum#82, count#83] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#84] -(73) HashAggregate [codegen id : 11] -Input [2]: [sum#79, count#80] +(78) HashAggregate [codegen id : 11] +Input [2]: [sum#82, count#83] Keys: [] -Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#82] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#82)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#83] +Functions [1]: [avg(netpaid#39)] +Aggregate Attributes [1]: [avg(netpaid#39)#85] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#85)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#86] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index 2d9ef020540b8..f60055c66c5cb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -13,7 +13,7 @@ WholeStageCodegen (11) WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + BroadcastHashJoin [c_birth_country,s_zip,upper(spark_catalog.default.customer_address.ca_country),ca_zip] Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -44,7 +44,13 @@ WholeStageCodegen (11) InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 + BroadcastExchange #12 + WholeStageCodegen (8) + Project [ca_state,ca_zip,ca_country] + Filter [ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name,s_store_name] #1 @@ -56,7 +62,7 @@ WholeStageCodegen (11) WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + BroadcastHashJoin [c_birth_country,s_zip,upper(spark_catalog.default.customer_address.ca_country),ca_zip] Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -112,7 +118,8 @@ WholeStageCodegen (11) InputAdapter BroadcastExchange #8 WholeStageCodegen (8) - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + Project [ca_state,ca_zip,ca_country] + Filter [ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt index 7db465072e4c3..9a243e2105c34 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt @@ -199,112 +199,112 @@ Right keys [1]: [ctr_customer_sk#33] Join condition: None (32) Project [codegen id : 17] -Output [14]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_state#34, ctr_total_return#35] +Output [15]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_state#34, ctr_total_return#35, cast(ctr_total_return#35 as decimal(24,7)) AS CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#36] Input [16]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_customer_sk#33, ctr_state#34, ctr_total_return#35] (33) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, wr_returned_date_sk#39] +Output [4]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39, wr_returned_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#39), dynamicpruningexpression(wr_returned_date_sk#39 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(wr_returned_date_sk#40), dynamicpruningexpression(wr_returned_date_sk#40 IN dynamicpruning#23)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 10] -Input [4]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, wr_returned_date_sk#39] +Input [4]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39, wr_returned_date_sk#40] (35) Filter [codegen id : 10] -Input [4]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, wr_returned_date_sk#39] -Condition : isnotnull(wr_returning_addr_sk#37) +Input [4]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39, wr_returned_date_sk#40] +Condition : isnotnull(wr_returning_addr_sk#38) (36) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#40] +Output [1]: [d_date_sk#41] (37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [wr_returned_date_sk#39] -Right keys [1]: [d_date_sk#40] +Left keys [1]: [wr_returned_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (38) Project [codegen id : 10] -Output [3]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38] -Input [5]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, wr_returned_date_sk#39, d_date_sk#40] +Output [3]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39] +Input [5]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39, wr_returned_date_sk#40, d_date_sk#41] (39) Exchange -Input [3]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38] -Arguments: hashpartitioning(wr_returning_addr_sk#37, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [3]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39] +Arguments: hashpartitioning(wr_returning_addr_sk#38, 5), ENSURE_REQUIREMENTS, [id=#42] (40) Sort [codegen id : 11] -Input [3]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38] -Arguments: [wr_returning_addr_sk#37 ASC NULLS FIRST], false, 0 +Input [3]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39] +Arguments: [wr_returning_addr_sk#38 ASC NULLS FIRST], false, 0 (41) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#42, ca_state#43] +Output [2]: [ca_address_sk#43, ca_state#44] (42) Sort [codegen id : 13] -Input [2]: [ca_address_sk#42, ca_state#43] -Arguments: [ca_address_sk#42 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#43, ca_state#44] +Arguments: [ca_address_sk#43 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 14] -Left keys [1]: [wr_returning_addr_sk#37] -Right keys [1]: [ca_address_sk#42] +Left keys [1]: [wr_returning_addr_sk#38] +Right keys [1]: [ca_address_sk#43] Join condition: None (44) Project [codegen id : 14] -Output [3]: [wr_returning_customer_sk#36, wr_return_amt#38, ca_state#43] -Input [5]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, ca_address_sk#42, ca_state#43] +Output [3]: [wr_returning_customer_sk#37, wr_return_amt#39, ca_state#44] +Input [5]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39, ca_address_sk#43, ca_state#44] (45) HashAggregate [codegen id : 14] -Input [3]: [wr_returning_customer_sk#36, wr_return_amt#38, ca_state#43] -Keys [2]: [wr_returning_customer_sk#36, ca_state#43] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#38))] -Aggregate Attributes [1]: [sum#44] -Results [3]: [wr_returning_customer_sk#36, ca_state#43, sum#45] +Input [3]: [wr_returning_customer_sk#37, wr_return_amt#39, ca_state#44] +Keys [2]: [wr_returning_customer_sk#37, ca_state#44] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#39))] +Aggregate Attributes [1]: [sum#45] +Results [3]: [wr_returning_customer_sk#37, ca_state#44, sum#46] (46) Exchange -Input [3]: [wr_returning_customer_sk#36, ca_state#43, sum#45] -Arguments: hashpartitioning(wr_returning_customer_sk#36, ca_state#43, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [wr_returning_customer_sk#37, ca_state#44, sum#46] +Arguments: hashpartitioning(wr_returning_customer_sk#37, ca_state#44, 5), ENSURE_REQUIREMENTS, [id=#47] (47) HashAggregate [codegen id : 15] -Input [3]: [wr_returning_customer_sk#36, ca_state#43, sum#45] -Keys [2]: [wr_returning_customer_sk#36, ca_state#43] -Functions [1]: [sum(UnscaledValue(wr_return_amt#38))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#38))#47] -Results [2]: [ca_state#43 AS ctr_state#34, MakeDecimal(sum(UnscaledValue(wr_return_amt#38))#47,17,2) AS ctr_total_return#35] +Input [3]: [wr_returning_customer_sk#37, ca_state#44, sum#46] +Keys [2]: [wr_returning_customer_sk#37, ca_state#44] +Functions [1]: [sum(UnscaledValue(wr_return_amt#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#39))#48] +Results [2]: [ca_state#44 AS ctr_state#34, MakeDecimal(sum(UnscaledValue(wr_return_amt#39))#48,17,2) AS ctr_total_return#35] (48) HashAggregate [codegen id : 15] Input [2]: [ctr_state#34, ctr_total_return#35] Keys [1]: [ctr_state#34] Functions [1]: [partial_avg(ctr_total_return#35)] -Aggregate Attributes [2]: [sum#48, count#49] -Results [3]: [ctr_state#34, sum#50, count#51] +Aggregate Attributes [2]: [sum#49, count#50] +Results [3]: [ctr_state#34, sum#51, count#52] (49) Exchange -Input [3]: [ctr_state#34, sum#50, count#51] -Arguments: hashpartitioning(ctr_state#34, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [3]: [ctr_state#34, sum#51, count#52] +Arguments: hashpartitioning(ctr_state#34, 5), ENSURE_REQUIREMENTS, [id=#53] (50) HashAggregate [codegen id : 16] -Input [3]: [ctr_state#34, sum#50, count#51] +Input [3]: [ctr_state#34, sum#51, count#52] Keys [1]: [ctr_state#34] Functions [1]: [avg(ctr_total_return#35)] -Aggregate Attributes [1]: [avg(ctr_total_return#35)#53] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#35)#53) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#54, ctr_state#34 AS ctr_state#34#55] +Aggregate Attributes [1]: [avg(ctr_total_return#35)#54] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#35)#54) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#55, ctr_state#34 AS ctr_state#34#56] (51) Filter [codegen id : 16] -Input [2]: [(avg(ctr_total_return) * 1.2)#54, ctr_state#34#55] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#54) +Input [2]: [(avg(ctr_total_return) * 1.2)#55, ctr_state#34#56] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#55) (52) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#54, ctr_state#34#55] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#56] +Input [2]: [(avg(ctr_total_return) * 1.2)#55, ctr_state#34#56] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#57] (53) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ctr_state#34] -Right keys [1]: [ctr_state#34#55] -Join condition: (cast(ctr_total_return#35 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#54) +Right keys [1]: [ctr_state#34#56] +Join condition: (CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#36 > (avg(ctr_total_return) * 1.2)#55) (54) Project [codegen id : 17] Output [13]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_total_return#35] -Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_state#34, ctr_total_return#35, (avg(ctr_total_return) * 1.2)#54, ctr_state#34#55] +Input [17]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_state#34, ctr_total_return#35, CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#36, (avg(ctr_total_return) * 1.2)#55, ctr_state#34#56] (55) TakeOrderedAndProject Input [13]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_total_return#35] @@ -321,27 +321,27 @@ BroadcastExchange (60) (56) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_year#57] +Output [2]: [d_date_sk#24, d_year#58] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#57] +Input [2]: [d_date_sk#24, d_year#58] (58) Filter [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#57] -Condition : ((isnotnull(d_year#57) AND (d_year#57 = 2002)) AND isnotnull(d_date_sk#24)) +Input [2]: [d_date_sk#24, d_year#58] +Condition : ((isnotnull(d_year#58) AND (d_year#58 = 2002)) AND isnotnull(d_date_sk#24)) (59) Project [codegen id : 1] Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#57] +Input [2]: [d_date_sk#24, d_year#58] (60) BroadcastExchange Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#59] -Subquery:2 Hosting operator id = 33 Hosting Expression = wr_returned_date_sk#39 IN dynamicpruning#23 +Subquery:2 Hosting operator id = 33 Hosting Expression = wr_returned_date_sk#40 IN dynamicpruning#23 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/simplified.txt index 6983e407ba891..2c51d1c8b349e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] WholeStageCodegen (17) Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] + BroadcastHashJoin [ctr_state,ctr_state,CAST(ctr1.ctr_total_return AS DECIMAL(24,7)),(avg(ctr_total_return) * 1.2)] Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_state,ctr_total_return] BroadcastHashJoin [c_customer_sk,ctr_customer_sk] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt index 066b0fa66bf33..163edf751ab1b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt @@ -1,53 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- * BroadcastHashJoin Inner BuildRight (47) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.web_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.customer_address (7) - : : +- BroadcastExchange (33) - : : +- * Filter (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * HashAggregate (28) - : : +- Exchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * Filter (19) - : : : : +- * ColumnarToRow (18) - : : : : +- Scan parquet default.web_returns (17) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : +- BroadcastExchange (39) - : +- * Filter (38) - : +- * ColumnarToRow (37) - : +- Scan parquet default.customer (36) - +- BroadcastExchange (46) - +- * Project (45) - +- * Filter (44) - +- * ColumnarToRow (43) - +- Scan parquet default.customer_address (42) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Project (17) + : : : +- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.web_returns (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.customer_address (7) + : : +- BroadcastExchange (34) + : : +- * Filter (33) + : : +- * HashAggregate (32) + : : +- Exchange (31) + : : +- * HashAggregate (30) + : : +- * HashAggregate (29) + : : +- Exchange (28) + : : +- * HashAggregate (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (23) + : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : :- * Filter (20) + : : : : +- * ColumnarToRow (19) + : : : : +- Scan parquet default.web_returns (18) + : : : +- ReusedExchange (21) + : : +- ReusedExchange (24) + : +- BroadcastExchange (40) + : +- * Filter (39) + : +- * ColumnarToRow (38) + : +- Scan parquet default.customer (37) + +- BroadcastExchange (47) + +- * Project (46) + +- * Filter (45) + +- * ColumnarToRow (44) + +- Scan parquet default.customer_address (43) (1) Scan parquet default.web_returns @@ -65,7 +66,7 @@ Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) -(4) ReusedExchange [Reuses operator id: 54] +(4) ReusedExchange [Reuses operator id: 55] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -126,192 +127,196 @@ Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#14, ca_state#8 AS ct Input [3]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16] Condition : isnotnull(ctr_total_return#16) -(17) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, wr_returned_date_sk#20] +(17) Project [codegen id : 11] +Output [4]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, cast(ctr_total_return#16 as decimal(24,7)) AS CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#17] +Input [3]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16] + +(18) Scan parquet default.web_returns +Output [4]: [wr_returning_customer_sk#18, wr_returning_addr_sk#19, wr_return_amt#20, wr_returned_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#20), dynamicpruningexpression(wr_returned_date_sk#20 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#21), dynamicpruningexpression(wr_returned_date_sk#21 IN dynamicpruning#5)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, wr_returned_date_sk#20] +(19) ColumnarToRow [codegen id : 6] +Input [4]: [wr_returning_customer_sk#18, wr_returning_addr_sk#19, wr_return_amt#20, wr_returned_date_sk#21] -(19) Filter [codegen id : 6] -Input [4]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, wr_returned_date_sk#20] -Condition : isnotnull(wr_returning_addr_sk#18) +(20) Filter [codegen id : 6] +Input [4]: [wr_returning_customer_sk#18, wr_returning_addr_sk#19, wr_return_amt#20, wr_returned_date_sk#21] +Condition : isnotnull(wr_returning_addr_sk#19) -(20) ReusedExchange [Reuses operator id: 54] -Output [1]: [d_date_sk#21] +(21) ReusedExchange [Reuses operator id: 55] +Output [1]: [d_date_sk#22] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#20] -Right keys [1]: [d_date_sk#21] +(22) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [wr_returned_date_sk#21] +Right keys [1]: [d_date_sk#22] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19] -Input [5]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, wr_returned_date_sk#20, d_date_sk#21] +(23) Project [codegen id : 6] +Output [3]: [wr_returning_customer_sk#18, wr_returning_addr_sk#19, wr_return_amt#20] +Input [5]: [wr_returning_customer_sk#18, wr_returning_addr_sk#19, wr_return_amt#20, wr_returned_date_sk#21, d_date_sk#22] -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#22, ca_state#23] +(24) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#23, ca_state#24] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#18] -Right keys [1]: [ca_address_sk#22] +(25) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [wr_returning_addr_sk#19] +Right keys [1]: [ca_address_sk#23] Join condition: None -(25) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#17, wr_return_amt#19, ca_state#23] -Input [5]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, ca_address_sk#22, ca_state#23] +(26) Project [codegen id : 6] +Output [3]: [wr_returning_customer_sk#18, wr_return_amt#20, ca_state#24] +Input [5]: [wr_returning_customer_sk#18, wr_returning_addr_sk#19, wr_return_amt#20, ca_address_sk#23, ca_state#24] -(26) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#17, wr_return_amt#19, ca_state#23] -Keys [2]: [wr_returning_customer_sk#17, ca_state#23] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#19))] -Aggregate Attributes [1]: [sum#24] -Results [3]: [wr_returning_customer_sk#17, ca_state#23, sum#25] +(27) HashAggregate [codegen id : 6] +Input [3]: [wr_returning_customer_sk#18, wr_return_amt#20, ca_state#24] +Keys [2]: [wr_returning_customer_sk#18, ca_state#24] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#20))] +Aggregate Attributes [1]: [sum#25] +Results [3]: [wr_returning_customer_sk#18, ca_state#24, sum#26] -(27) Exchange -Input [3]: [wr_returning_customer_sk#17, ca_state#23, sum#25] -Arguments: hashpartitioning(wr_returning_customer_sk#17, ca_state#23, 5), ENSURE_REQUIREMENTS, [id=#26] - -(28) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#17, ca_state#23, sum#25] -Keys [2]: [wr_returning_customer_sk#17, ca_state#23] -Functions [1]: [sum(UnscaledValue(wr_return_amt#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#19))#27] -Results [2]: [ca_state#23 AS ctr_state#15, MakeDecimal(sum(UnscaledValue(wr_return_amt#19))#27,17,2) AS ctr_total_return#16] +(28) Exchange +Input [3]: [wr_returning_customer_sk#18, ca_state#24, sum#26] +Arguments: hashpartitioning(wr_returning_customer_sk#18, ca_state#24, 5), ENSURE_REQUIREMENTS, [id=#27] (29) HashAggregate [codegen id : 7] +Input [3]: [wr_returning_customer_sk#18, ca_state#24, sum#26] +Keys [2]: [wr_returning_customer_sk#18, ca_state#24] +Functions [1]: [sum(UnscaledValue(wr_return_amt#20))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#20))#28] +Results [2]: [ca_state#24 AS ctr_state#15, MakeDecimal(sum(UnscaledValue(wr_return_amt#20))#28,17,2) AS ctr_total_return#16] + +(30) HashAggregate [codegen id : 7] Input [2]: [ctr_state#15, ctr_total_return#16] Keys [1]: [ctr_state#15] Functions [1]: [partial_avg(ctr_total_return#16)] -Aggregate Attributes [2]: [sum#28, count#29] -Results [3]: [ctr_state#15, sum#30, count#31] +Aggregate Attributes [2]: [sum#29, count#30] +Results [3]: [ctr_state#15, sum#31, count#32] -(30) Exchange -Input [3]: [ctr_state#15, sum#30, count#31] -Arguments: hashpartitioning(ctr_state#15, 5), ENSURE_REQUIREMENTS, [id=#32] +(31) Exchange +Input [3]: [ctr_state#15, sum#31, count#32] +Arguments: hashpartitioning(ctr_state#15, 5), ENSURE_REQUIREMENTS, [id=#33] -(31) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#15, sum#30, count#31] +(32) HashAggregate [codegen id : 8] +Input [3]: [ctr_state#15, sum#31, count#32] Keys [1]: [ctr_state#15] Functions [1]: [avg(ctr_total_return#16)] -Aggregate Attributes [1]: [avg(ctr_total_return#16)#33] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#16)#33) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#34, ctr_state#15 AS ctr_state#15#35] +Aggregate Attributes [1]: [avg(ctr_total_return#16)#34] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#16)#34) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#35, ctr_state#15 AS ctr_state#15#36] -(32) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#34) +(33) Filter [codegen id : 8] +Input [2]: [(avg(ctr_total_return) * 1.2)#35, ctr_state#15#36] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#35) -(33) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#36] +(34) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#35, ctr_state#15#36] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#37] -(34) BroadcastHashJoin [codegen id : 11] +(35) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#15] -Right keys [1]: [ctr_state#15#35] -Join condition: (cast(ctr_total_return#16 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#34) +Right keys [1]: [ctr_state#15#36] +Join condition: (CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#17 > (avg(ctr_total_return) * 1.2)#35) -(35) Project [codegen id : 11] +(36) Project [codegen id : 11] Output [2]: [ctr_customer_sk#14, ctr_total_return#16] -Input [5]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, (avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] +Input [6]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#17, (avg(ctr_total_return) * 1.2)#35, ctr_state#15#36] -(36) Scan parquet default.customer -Output [14]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] +(37) Scan parquet default.customer +Output [14]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] +(38) ColumnarToRow [codegen id : 9] +Input [14]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51] -(38) Filter [codegen id : 9] -Input [14]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] -Condition : (isnotnull(c_customer_sk#37) AND isnotnull(c_current_addr_sk#39)) +(39) Filter [codegen id : 9] +Input [14]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51] +Condition : (isnotnull(c_customer_sk#38) AND isnotnull(c_current_addr_sk#40)) -(39) BroadcastExchange -Input [14]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +(40) BroadcastExchange +Input [14]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] -(40) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#14] -Right keys [1]: [c_customer_sk#37] +Right keys [1]: [c_customer_sk#38] Join condition: None -(41) Project [codegen id : 11] -Output [14]: [ctr_total_return#16, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] -Input [16]: [ctr_customer_sk#14, ctr_total_return#16, c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] +(42) Project [codegen id : 11] +Output [14]: [ctr_total_return#16, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51] +Input [16]: [ctr_customer_sk#14, ctr_total_return#16, c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51] -(42) Scan parquet default.customer_address -Output [2]: [ca_address_sk#52, ca_state#53] +(43) Scan parquet default.customer_address +Output [2]: [ca_address_sk#53, ca_state#54] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 10] -Input [2]: [ca_address_sk#52, ca_state#53] +(44) ColumnarToRow [codegen id : 10] +Input [2]: [ca_address_sk#53, ca_state#54] -(44) Filter [codegen id : 10] -Input [2]: [ca_address_sk#52, ca_state#53] -Condition : ((isnotnull(ca_state#53) AND (ca_state#53 = GA)) AND isnotnull(ca_address_sk#52)) +(45) Filter [codegen id : 10] +Input [2]: [ca_address_sk#53, ca_state#54] +Condition : ((isnotnull(ca_state#54) AND (ca_state#54 = GA)) AND isnotnull(ca_address_sk#53)) -(45) Project [codegen id : 10] -Output [1]: [ca_address_sk#52] -Input [2]: [ca_address_sk#52, ca_state#53] +(46) Project [codegen id : 10] +Output [1]: [ca_address_sk#53] +Input [2]: [ca_address_sk#53, ca_state#54] -(46) BroadcastExchange -Input [1]: [ca_address_sk#52] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +(47) BroadcastExchange +Input [1]: [ca_address_sk#53] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#55] -(47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#39] -Right keys [1]: [ca_address_sk#52] +(48) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#40] +Right keys [1]: [ca_address_sk#53] Join condition: None -(48) Project [codegen id : 11] -Output [13]: [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50, ctr_total_return#16] -Input [15]: [ctr_total_return#16, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50, ca_address_sk#52] +(49) Project [codegen id : 11] +Output [13]: [c_customer_id#39, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51, ctr_total_return#16] +Input [15]: [ctr_total_return#16, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51, ca_address_sk#53] -(49) TakeOrderedAndProject -Input [13]: [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50, ctr_total_return#16] -Arguments: 100, [c_customer_id#38 ASC NULLS FIRST, c_salutation#40 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, c_last_name#42 ASC NULLS FIRST, c_preferred_cust_flag#43 ASC NULLS FIRST, c_birth_day#44 ASC NULLS FIRST, c_birth_month#45 ASC NULLS FIRST, c_birth_year#46 ASC NULLS FIRST, c_birth_country#47 ASC NULLS FIRST, c_login#48 ASC NULLS FIRST, c_email_address#49 ASC NULLS FIRST, c_last_review_date#50 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50, ctr_total_return#16] +(50) TakeOrderedAndProject +Input [13]: [c_customer_id#39, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51, ctr_total_return#16] +Arguments: 100, [c_customer_id#39 ASC NULLS FIRST, c_salutation#41 ASC NULLS FIRST, c_first_name#42 ASC NULLS FIRST, c_last_name#43 ASC NULLS FIRST, c_preferred_cust_flag#44 ASC NULLS FIRST, c_birth_day#45 ASC NULLS FIRST, c_birth_month#46 ASC NULLS FIRST, c_birth_year#47 ASC NULLS FIRST, c_birth_country#48 ASC NULLS FIRST, c_login#49 ASC NULLS FIRST, c_email_address#50 ASC NULLS FIRST, c_last_review_date#51 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#39, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51, ctr_total_return#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (54) -+- * Project (53) - +- * Filter (52) - +- * ColumnarToRow (51) - +- Scan parquet default.date_dim (50) +BroadcastExchange (55) ++- * Project (54) + +- * Filter (53) + +- * ColumnarToRow (52) + +- Scan parquet default.date_dim (51) -(50) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#55] +(51) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_year#56] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(51) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#55] +(52) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#56] -(52) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#55] -Condition : ((isnotnull(d_year#55) AND (d_year#55 = 2002)) AND isnotnull(d_date_sk#6)) +(53) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#56] +Condition : ((isnotnull(d_year#56) AND (d_year#56 = 2002)) AND isnotnull(d_date_sk#6)) -(53) Project [codegen id : 1] +(54) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#55] +Input [2]: [d_date_sk#6, d_year#56] -(54) BroadcastExchange +(55) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#57] -Subquery:2 Hosting operator id = 17 Hosting Expression = wr_returned_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 Hosting Expression = wr_returned_date_sk#21 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt index 884dce2c6583a..dc3b631040b2d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -5,38 +5,39 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [wr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_returning_addr_sk,wr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [ca_address_sk,ca_state] + BroadcastHashJoin [ctr_state,ctr_state,CAST(ctr1.ctr_total_return AS DECIMAL(24,7)),(avg(ctr_total_return) * 1.2)] + Project [ctr_customer_sk,ctr_state,ctr_total_return] + Filter [ctr_total_return] + HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] + InputAdapter + Exchange [wr_returning_customer_sk,ca_state] #1 + WholeStageCodegen (3) + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + Project [wr_returning_customer_sk,wr_return_amt,ca_state] + BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + Filter [wr_returning_addr_sk,wr_returning_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] + Scan parquet default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Filter [ca_address_sk,ca_state] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt index a9569e31b039f..cba3fdedc086c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt @@ -356,261 +356,261 @@ Right keys [1]: [ca_county#42] Join condition: None (54) Project [codegen id : 42] -Output [5]: [store_sales#16, store_sales#31, ca_county#42, d_year#38, store_sales#47] +Output [7]: [store_sales#16, store_sales#31, ca_county#42, d_year#38, store_sales#47, CASE WHEN (store_sales#47 > 0.00) THEN CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#47)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#49, CASE WHEN (store_sales#16 > 0.00) THEN CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#50] Input [6]: [ca_county#10, store_sales#16, store_sales#31, ca_county#42, d_year#38, store_sales#47] (55) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Output [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] (57) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_bill_addr_sk#49) +Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_bill_addr_sk#51) (58) ReusedExchange [Reuses operator id: 122] -Output [3]: [d_date_sk#52, d_year#53, d_qoy#54] +Output [3]: [d_date_sk#54, d_year#55, d_qoy#56] (59) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#52] +Left keys [1]: [ws_sold_date_sk#53] +Right keys [1]: [d_date_sk#54] Join condition: None (60) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54] -Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51, d_date_sk#52, d_year#53, d_qoy#54] +Output [4]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#55, d_qoy#56] +Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53, d_date_sk#54, d_year#55, d_qoy#56] (61) Exchange -Input [4]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54] -Arguments: hashpartitioning(ws_bill_addr_sk#49, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#55, d_qoy#56] +Arguments: hashpartitioning(ws_bill_addr_sk#51, 5), ENSURE_REQUIREMENTS, [id=#57] (62) Sort [codegen id : 23] -Input [4]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54] -Arguments: [ws_bill_addr_sk#49 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#55, d_qoy#56] +Arguments: [ws_bill_addr_sk#51 ASC NULLS FIRST], false, 0 (63) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#56, ca_county#57] +Output [2]: [ca_address_sk#58, ca_county#59] (64) Sort [codegen id : 25] -Input [2]: [ca_address_sk#56, ca_county#57] -Arguments: [ca_address_sk#56 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#58, ca_county#59] +Arguments: [ca_address_sk#58 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] -Left keys [1]: [ws_bill_addr_sk#49] -Right keys [1]: [ca_address_sk#56] +Left keys [1]: [ws_bill_addr_sk#51] +Right keys [1]: [ca_address_sk#58] Join condition: None (66) Project [codegen id : 26] -Output [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#57] -Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_address_sk#56, ca_county#57] +Output [4]: [ws_ext_sales_price#52, d_year#55, d_qoy#56, ca_county#59] +Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#55, d_qoy#56, ca_address_sk#58, ca_county#59] (67) HashAggregate [codegen id : 26] -Input [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#57] -Keys [3]: [ca_county#57, d_qoy#54, d_year#53] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#50))] -Aggregate Attributes [1]: [sum#58] -Results [4]: [ca_county#57, d_qoy#54, d_year#53, sum#59] +Input [4]: [ws_ext_sales_price#52, d_year#55, d_qoy#56, ca_county#59] +Keys [3]: [ca_county#59, d_qoy#56, d_year#55] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#52))] +Aggregate Attributes [1]: [sum#60] +Results [4]: [ca_county#59, d_qoy#56, d_year#55, sum#61] (68) Exchange -Input [4]: [ca_county#57, d_qoy#54, d_year#53, sum#59] -Arguments: hashpartitioning(ca_county#57, d_qoy#54, d_year#53, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [4]: [ca_county#59, d_qoy#56, d_year#55, sum#61] +Arguments: hashpartitioning(ca_county#59, d_qoy#56, d_year#55, 5), ENSURE_REQUIREMENTS, [id=#62] (69) HashAggregate [codegen id : 41] -Input [4]: [ca_county#57, d_qoy#54, d_year#53, sum#59] -Keys [3]: [ca_county#57, d_qoy#54, d_year#53] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#50))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#50))#61] -Results [2]: [ca_county#57, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#50))#61,17,2) AS web_sales#62] +Input [4]: [ca_county#59, d_qoy#56, d_year#55, sum#61] +Keys [3]: [ca_county#59, d_qoy#56, d_year#55] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#52))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#52))#63] +Results [2]: [ca_county#59, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#52))#63,17,2) AS web_sales#64] (70) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Output [3]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, ws_sold_date_sk#67] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#67), dynamicpruningexpression(ws_sold_date_sk#67 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (71) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Input [3]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, ws_sold_date_sk#67] (72) Filter [codegen id : 28] -Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_bill_addr_sk#63) +Input [3]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, ws_sold_date_sk#67] +Condition : isnotnull(ws_bill_addr_sk#65) (73) ReusedExchange [Reuses operator id: 114] -Output [3]: [d_date_sk#66, d_year#67, d_qoy#68] +Output [3]: [d_date_sk#68, d_year#69, d_qoy#70] (74) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#66] +Left keys [1]: [ws_sold_date_sk#67] +Right keys [1]: [d_date_sk#68] Join condition: None (75) Project [codegen id : 28] -Output [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68] -Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65, d_date_sk#66, d_year#67, d_qoy#68] +Output [4]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, d_year#69, d_qoy#70] +Input [6]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, ws_sold_date_sk#67, d_date_sk#68, d_year#69, d_qoy#70] (76) Exchange -Input [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68] -Arguments: hashpartitioning(ws_bill_addr_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [4]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, d_year#69, d_qoy#70] +Arguments: hashpartitioning(ws_bill_addr_sk#65, 5), ENSURE_REQUIREMENTS, [id=#71] (77) Sort [codegen id : 29] -Input [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68] -Arguments: [ws_bill_addr_sk#63 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, d_year#69, d_qoy#70] +Arguments: [ws_bill_addr_sk#65 ASC NULLS FIRST], false, 0 (78) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#70, ca_county#71] +Output [2]: [ca_address_sk#72, ca_county#73] (79) Sort [codegen id : 31] -Input [2]: [ca_address_sk#70, ca_county#71] -Arguments: [ca_address_sk#70 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#72, ca_county#73] +Arguments: [ca_address_sk#72 ASC NULLS FIRST], false, 0 (80) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_addr_sk#63] -Right keys [1]: [ca_address_sk#70] +Left keys [1]: [ws_bill_addr_sk#65] +Right keys [1]: [ca_address_sk#72] Join condition: None (81) Project [codegen id : 32] -Output [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#71] -Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_address_sk#70, ca_county#71] +Output [4]: [ws_ext_sales_price#66, d_year#69, d_qoy#70, ca_county#73] +Input [6]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, d_year#69, d_qoy#70, ca_address_sk#72, ca_county#73] (82) HashAggregate [codegen id : 32] -Input [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#71] -Keys [3]: [ca_county#71, d_qoy#68, d_year#67] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] -Aggregate Attributes [1]: [sum#72] -Results [4]: [ca_county#71, d_qoy#68, d_year#67, sum#73] +Input [4]: [ws_ext_sales_price#66, d_year#69, d_qoy#70, ca_county#73] +Keys [3]: [ca_county#73, d_qoy#70, d_year#69] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#66))] +Aggregate Attributes [1]: [sum#74] +Results [4]: [ca_county#73, d_qoy#70, d_year#69, sum#75] (83) Exchange -Input [4]: [ca_county#71, d_qoy#68, d_year#67, sum#73] -Arguments: hashpartitioning(ca_county#71, d_qoy#68, d_year#67, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [4]: [ca_county#73, d_qoy#70, d_year#69, sum#75] +Arguments: hashpartitioning(ca_county#73, d_qoy#70, d_year#69, 5), ENSURE_REQUIREMENTS, [id=#76] (84) HashAggregate [codegen id : 33] -Input [4]: [ca_county#71, d_qoy#68, d_year#67, sum#73] -Keys [3]: [ca_county#71, d_qoy#68, d_year#67] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#64))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#64))#75] -Results [2]: [ca_county#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#75,17,2) AS web_sales#76] +Input [4]: [ca_county#73, d_qoy#70, d_year#69, sum#75] +Keys [3]: [ca_county#73, d_qoy#70, d_year#69] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#66))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#66))#77] +Results [2]: [ca_county#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#77,17,2) AS web_sales#78] (85) BroadcastExchange -Input [2]: [ca_county#71, web_sales#76] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#77] +Input [2]: [ca_county#73, web_sales#78] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#79] (86) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#57] -Right keys [1]: [ca_county#71] +Left keys [1]: [ca_county#59] +Right keys [1]: [ca_county#73] Join condition: None (87) Project [codegen id : 41] -Output [3]: [ca_county#57, web_sales#62, web_sales#76] -Input [4]: [ca_county#57, web_sales#62, ca_county#71, web_sales#76] +Output [3]: [ca_county#59, web_sales#64, web_sales#78] +Input [4]: [ca_county#59, web_sales#64, ca_county#73, web_sales#78] (88) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, ws_sold_date_sk#80] +Output [3]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, ws_sold_date_sk#82] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(ws_sold_date_sk#82), dynamicpruningexpression(ws_sold_date_sk#82 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (89) ColumnarToRow [codegen id : 35] -Input [3]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, ws_sold_date_sk#80] +Input [3]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, ws_sold_date_sk#82] (90) Filter [codegen id : 35] -Input [3]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, ws_sold_date_sk#80] -Condition : isnotnull(ws_bill_addr_sk#78) +Input [3]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, ws_sold_date_sk#82] +Condition : isnotnull(ws_bill_addr_sk#80) (91) ReusedExchange [Reuses operator id: 118] -Output [3]: [d_date_sk#81, d_year#82, d_qoy#83] +Output [3]: [d_date_sk#83, d_year#84, d_qoy#85] (92) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_sold_date_sk#80] -Right keys [1]: [d_date_sk#81] +Left keys [1]: [ws_sold_date_sk#82] +Right keys [1]: [d_date_sk#83] Join condition: None (93) Project [codegen id : 35] -Output [4]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, d_year#82, d_qoy#83] -Input [6]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, ws_sold_date_sk#80, d_date_sk#81, d_year#82, d_qoy#83] +Output [4]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, d_year#84, d_qoy#85] +Input [6]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, ws_sold_date_sk#82, d_date_sk#83, d_year#84, d_qoy#85] (94) Exchange -Input [4]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, d_year#82, d_qoy#83] -Arguments: hashpartitioning(ws_bill_addr_sk#78, 5), ENSURE_REQUIREMENTS, [id=#84] +Input [4]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, d_year#84, d_qoy#85] +Arguments: hashpartitioning(ws_bill_addr_sk#80, 5), ENSURE_REQUIREMENTS, [id=#86] (95) Sort [codegen id : 36] -Input [4]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, d_year#82, d_qoy#83] -Arguments: [ws_bill_addr_sk#78 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, d_year#84, d_qoy#85] +Arguments: [ws_bill_addr_sk#80 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#85, ca_county#86] +Output [2]: [ca_address_sk#87, ca_county#88] (97) Sort [codegen id : 38] -Input [2]: [ca_address_sk#85, ca_county#86] -Arguments: [ca_address_sk#85 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#87, ca_county#88] +Arguments: [ca_address_sk#87 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 39] -Left keys [1]: [ws_bill_addr_sk#78] -Right keys [1]: [ca_address_sk#85] +Left keys [1]: [ws_bill_addr_sk#80] +Right keys [1]: [ca_address_sk#87] Join condition: None (99) Project [codegen id : 39] -Output [4]: [ws_ext_sales_price#79, d_year#82, d_qoy#83, ca_county#86] -Input [6]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, d_year#82, d_qoy#83, ca_address_sk#85, ca_county#86] +Output [4]: [ws_ext_sales_price#81, d_year#84, d_qoy#85, ca_county#88] +Input [6]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, d_year#84, d_qoy#85, ca_address_sk#87, ca_county#88] (100) HashAggregate [codegen id : 39] -Input [4]: [ws_ext_sales_price#79, d_year#82, d_qoy#83, ca_county#86] -Keys [3]: [ca_county#86, d_qoy#83, d_year#82] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#79))] -Aggregate Attributes [1]: [sum#87] -Results [4]: [ca_county#86, d_qoy#83, d_year#82, sum#88] +Input [4]: [ws_ext_sales_price#81, d_year#84, d_qoy#85, ca_county#88] +Keys [3]: [ca_county#88, d_qoy#85, d_year#84] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#81))] +Aggregate Attributes [1]: [sum#89] +Results [4]: [ca_county#88, d_qoy#85, d_year#84, sum#90] (101) Exchange -Input [4]: [ca_county#86, d_qoy#83, d_year#82, sum#88] -Arguments: hashpartitioning(ca_county#86, d_qoy#83, d_year#82, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [4]: [ca_county#88, d_qoy#85, d_year#84, sum#90] +Arguments: hashpartitioning(ca_county#88, d_qoy#85, d_year#84, 5), ENSURE_REQUIREMENTS, [id=#91] (102) HashAggregate [codegen id : 40] -Input [4]: [ca_county#86, d_qoy#83, d_year#82, sum#88] -Keys [3]: [ca_county#86, d_qoy#83, d_year#82] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#79))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#79))#90] -Results [2]: [ca_county#86, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#90,17,2) AS web_sales#91] +Input [4]: [ca_county#88, d_qoy#85, d_year#84, sum#90] +Keys [3]: [ca_county#88, d_qoy#85, d_year#84] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#81))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#81))#92] +Results [2]: [ca_county#88, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#81))#92,17,2) AS web_sales#93] (103) BroadcastExchange -Input [2]: [ca_county#86, web_sales#91] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#92] +Input [2]: [ca_county#88, web_sales#93] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#94] (104) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#57] -Right keys [1]: [ca_county#86] +Left keys [1]: [ca_county#59] +Right keys [1]: [ca_county#88] Join condition: None (105) Project [codegen id : 41] -Output [4]: [ca_county#57, web_sales#62, web_sales#76, web_sales#91] -Input [5]: [ca_county#57, web_sales#62, web_sales#76, ca_county#86, web_sales#91] +Output [6]: [ca_county#59, web_sales#64, web_sales#78, web_sales#93, CASE WHEN (web_sales#64 > 0.00) THEN CheckOverflow((promote_precision(web_sales#78) / promote_precision(web_sales#64)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (ws1.web_sales > 0.00BD) THEN (ws2.web_sales / ws1.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#95, CASE WHEN (web_sales#78 > 0.00) THEN CheckOverflow((promote_precision(web_sales#93) / promote_precision(web_sales#78)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (ws2.web_sales > 0.00BD) THEN (ws3.web_sales / ws2.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#96] +Input [5]: [ca_county#59, web_sales#64, web_sales#78, ca_county#88, web_sales#93] (106) BroadcastExchange -Input [4]: [ca_county#57, web_sales#62, web_sales#76, web_sales#91] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#93] +Input [6]: [ca_county#59, web_sales#64, web_sales#78, web_sales#93, CASE WHEN (ws1.web_sales > 0.00BD) THEN (ws2.web_sales / ws1.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#95, CASE WHEN (ws2.web_sales > 0.00BD) THEN (ws3.web_sales / ws2.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#96] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#97] (107) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#42] -Right keys [1]: [ca_county#57] -Join condition: ((CASE WHEN (web_sales#62 > 0.00) THEN CheckOverflow((promote_precision(web_sales#76) / promote_precision(web_sales#62)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#47 > 0.00) THEN CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#47)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#76 > 0.00) THEN CheckOverflow((promote_precision(web_sales#91) / promote_precision(web_sales#76)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#16 > 0.00) THEN CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) ELSE null END)) +Right keys [1]: [ca_county#59] +Join condition: ((CASE WHEN (ws1.web_sales > 0.00BD) THEN (ws2.web_sales / ws1.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#95 > CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#49) AND (CASE WHEN (ws2.web_sales > 0.00BD) THEN (ws3.web_sales / ws2.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#96 > CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#50)) (108) Project [codegen id : 42] -Output [6]: [ca_county#42, d_year#38, CheckOverflow((promote_precision(web_sales#76) / promote_precision(web_sales#62)), DecimalType(37,20), true) AS web_q1_q2_increase#94, CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#47)), DecimalType(37,20), true) AS store_q1_q2_increase#95, CheckOverflow((promote_precision(web_sales#91) / promote_precision(web_sales#76)), DecimalType(37,20), true) AS web_q2_q3_increase#96, CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q2_q3_increase#97] -Input [9]: [store_sales#16, store_sales#31, ca_county#42, d_year#38, store_sales#47, ca_county#57, web_sales#62, web_sales#76, web_sales#91] +Output [6]: [ca_county#42, d_year#38, CheckOverflow((promote_precision(web_sales#78) / promote_precision(web_sales#64)), DecimalType(37,20), true) AS web_q1_q2_increase#98, CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#47)), DecimalType(37,20), true) AS store_q1_q2_increase#99, CheckOverflow((promote_precision(web_sales#93) / promote_precision(web_sales#78)), DecimalType(37,20), true) AS web_q2_q3_increase#100, CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q2_q3_increase#101] +Input [13]: [store_sales#16, store_sales#31, ca_county#42, d_year#38, store_sales#47, CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#49, CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#50, ca_county#59, web_sales#64, web_sales#78, web_sales#93, CASE WHEN (ws1.web_sales > 0.00BD) THEN (ws2.web_sales / ws1.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#95, CASE WHEN (ws2.web_sales > 0.00BD) THEN (ws3.web_sales / ws2.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#96] (109) Exchange -Input [6]: [ca_county#42, d_year#38, web_q1_q2_increase#94, store_q1_q2_increase#95, web_q2_q3_increase#96, store_q2_q3_increase#97] -Arguments: rangepartitioning(ca_county#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [6]: [ca_county#42, d_year#38, web_q1_q2_increase#98, store_q1_q2_increase#99, web_q2_q3_increase#100, store_q2_q3_increase#101] +Arguments: rangepartitioning(ca_county#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#102] (110) Sort [codegen id : 43] -Input [6]: [ca_county#42, d_year#38, web_q1_q2_increase#94, store_q1_q2_increase#95, web_q2_q3_increase#96, store_q2_q3_increase#97] +Input [6]: [ca_county#42, d_year#38, web_q1_q2_increase#98, store_q1_q2_increase#99, web_q2_q3_increase#100, store_q2_q3_increase#101] Arguments: [ca_county#42 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -638,7 +638,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A (114) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#19 IN dynamicpruning#20 BroadcastExchange (118) @@ -663,7 +663,7 @@ Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 3) (118) BroadcastExchange Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#100] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#104] Subquery:3 Hosting operator id = 37 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 BroadcastExchange (122) @@ -688,12 +688,12 @@ Condition : ((((isnotnull(d_qoy#39) AND isnotnull(d_year#38)) AND (d_qoy#39 = 1) (122) BroadcastExchange Input [3]: [d_date_sk#37, d_year#38, d_qoy#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#101] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#105] -Subquery:4 Hosting operator id = 55 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#36 +Subquery:4 Hosting operator id = 55 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#36 -Subquery:5 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#4 +Subquery:5 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#67 IN dynamicpruning#4 -Subquery:6 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#20 +Subquery:6 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#82 IN dynamicpruning#20 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt index ea20602bf0c97..d4620233dc820 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt @@ -4,7 +4,7 @@ WholeStageCodegen (43) Exchange [ca_county] #1 WholeStageCodegen (42) Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county,CASE WHEN (ws1.web_sales > 0.00BD) THEN (ws2.web_sales / ws1.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END,CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END,CASE WHEN (ws2.web_sales > 0.00BD) THEN (ws3.web_sales / ws2.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END,CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END] Project [store_sales,store_sales,ca_county,d_year,store_sales] BroadcastHashJoin [ca_county,ca_county] Project [ca_county,store_sales,store_sales] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt index 9f758c9c4cf6b..2cd869cf5e172 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt @@ -1,94 +1,95 @@ == Physical Plan == -* Sort (90) -+- Exchange (89) - +- * Project (88) - +- * BroadcastHashJoin Inner BuildRight (87) - :- * Project (73) - : +- * BroadcastHashJoin Inner BuildRight (72) - : :- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * BroadcastHashJoin Inner BuildRight (29) - : : : : :- * HashAggregate (15) - : : : : : +- Exchange (14) - : : : : : +- * HashAggregate (13) - : : : : : +- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Project (6) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet default.store_sales (1) - : : : : : : +- ReusedExchange (4) - : : : : : +- BroadcastExchange (10) - : : : : : +- * Filter (9) - : : : : : +- * ColumnarToRow (8) - : : : : : +- Scan parquet default.customer_address (7) - : : : : +- BroadcastExchange (28) - : : : : +- * HashAggregate (27) - : : : : +- Exchange (26) - : : : : +- * HashAggregate (25) - : : : : +- * Project (24) - : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : :- * Project (21) - : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : :- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.store_sales (16) - : : : : : +- ReusedExchange (19) - : : : : +- ReusedExchange (22) - : : : +- BroadcastExchange (42) - : : : +- * HashAggregate (41) - : : : +- Exchange (40) - : : : +- * HashAggregate (39) - : : : +- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : :- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Filter (32) - : : : : : +- * ColumnarToRow (31) - : : : : : +- Scan parquet default.store_sales (30) - : : : : +- ReusedExchange (33) - : : : +- ReusedExchange (36) - : : +- BroadcastExchange (57) - : : +- * HashAggregate (56) - : : +- Exchange (55) - : : +- * HashAggregate (54) - : : +- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Filter (47) - : : : : +- * ColumnarToRow (46) - : : : : +- Scan parquet default.web_sales (45) - : : : +- ReusedExchange (48) - : : +- ReusedExchange (51) - : +- BroadcastExchange (71) - : +- * HashAggregate (70) - : +- Exchange (69) - : +- * HashAggregate (68) - : +- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Filter (61) - : : : +- * ColumnarToRow (60) - : : : +- Scan parquet default.web_sales (59) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- BroadcastExchange (86) - +- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- * Project (82) - +- * BroadcastHashJoin Inner BuildRight (81) - :- * Project (79) - : +- * BroadcastHashJoin Inner BuildRight (78) - : :- * Filter (76) - : : +- * ColumnarToRow (75) - : : +- Scan parquet default.web_sales (74) - : +- ReusedExchange (77) - +- ReusedExchange (80) +* Sort (91) ++- Exchange (90) + +- * Project (89) + +- * BroadcastHashJoin Inner BuildRight (88) + :- * Project (74) + : +- * BroadcastHashJoin Inner BuildRight (73) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * BroadcastHashJoin Inner BuildRight (29) + : : : : :- * HashAggregate (15) + : : : : : +- Exchange (14) + : : : : : +- * HashAggregate (13) + : : : : : +- * Project (12) + : : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : : :- * Project (6) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : : : :- * Filter (3) + : : : : : : : +- * ColumnarToRow (2) + : : : : : : : +- Scan parquet default.store_sales (1) + : : : : : : +- ReusedExchange (4) + : : : : : +- BroadcastExchange (10) + : : : : : +- * Filter (9) + : : : : : +- * ColumnarToRow (8) + : : : : : +- Scan parquet default.customer_address (7) + : : : : +- BroadcastExchange (28) + : : : : +- * HashAggregate (27) + : : : : +- Exchange (26) + : : : : +- * HashAggregate (25) + : : : : +- * Project (24) + : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : :- * Project (21) + : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : :- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.store_sales (16) + : : : : : +- ReusedExchange (19) + : : : : +- ReusedExchange (22) + : : : +- BroadcastExchange (42) + : : : +- * HashAggregate (41) + : : : +- Exchange (40) + : : : +- * HashAggregate (39) + : : : +- * Project (38) + : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : :- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Filter (32) + : : : : : +- * ColumnarToRow (31) + : : : : : +- Scan parquet default.store_sales (30) + : : : : +- ReusedExchange (33) + : : : +- ReusedExchange (36) + : : +- BroadcastExchange (57) + : : +- * HashAggregate (56) + : : +- Exchange (55) + : : +- * HashAggregate (54) + : : +- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * Filter (47) + : : : : +- * ColumnarToRow (46) + : : : : +- Scan parquet default.web_sales (45) + : : : +- ReusedExchange (48) + : : +- ReusedExchange (51) + : +- BroadcastExchange (72) + : +- * HashAggregate (71) + : +- Exchange (70) + : +- * HashAggregate (69) + : +- * Project (68) + : +- * BroadcastHashJoin Inner BuildRight (67) + : :- * Project (65) + : : +- * BroadcastHashJoin Inner BuildRight (64) + : : :- * Filter (62) + : : : +- * ColumnarToRow (61) + : : : +- Scan parquet default.web_sales (60) + : : +- ReusedExchange (63) + : +- ReusedExchange (66) + +- BroadcastExchange (87) + +- * HashAggregate (86) + +- Exchange (85) + +- * HashAggregate (84) + +- * Project (83) + +- * BroadcastHashJoin Inner BuildRight (82) + :- * Project (80) + : +- * BroadcastHashJoin Inner BuildRight (79) + : :- * Filter (77) + : : +- * ColumnarToRow (76) + : : +- Scan parquet default.web_sales (75) + : +- ReusedExchange (78) + +- ReusedExchange (81) (1) Scan parquet default.store_sales @@ -106,7 +107,7 @@ Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_addr_sk#1) -(4) ReusedExchange [Reuses operator id: 94] +(4) ReusedExchange [Reuses operator id: 95] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] (5) BroadcastHashJoin [codegen id : 3] @@ -178,7 +179,7 @@ Input [3]: [ss_addr_sk#16, ss_ext_sales_price#17, ss_sold_date_sk#18] Input [3]: [ss_addr_sk#16, ss_ext_sales_price#17, ss_sold_date_sk#18] Condition : isnotnull(ss_addr_sk#16) -(19) ReusedExchange [Reuses operator id: 98] +(19) ReusedExchange [Reuses operator id: 99] Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] (20) BroadcastHashJoin [codegen id : 6] @@ -244,7 +245,7 @@ Input [3]: [ss_addr_sk#31, ss_ext_sales_price#32, ss_sold_date_sk#33] Input [3]: [ss_addr_sk#31, ss_ext_sales_price#32, ss_sold_date_sk#33] Condition : isnotnull(ss_addr_sk#31) -(33) ReusedExchange [Reuses operator id: 102] +(33) ReusedExchange [Reuses operator id: 103] Output [3]: [d_date_sk#35, d_year#36, d_qoy#37] (34) BroadcastHashJoin [codegen id : 10] @@ -314,7 +315,7 @@ Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] Condition : isnotnull(ws_bill_addr_sk#46) -(48) ReusedExchange [Reuses operator id: 94] +(48) ReusedExchange [Reuses operator id: 95] Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] (49) BroadcastHashJoin [codegen id : 14] @@ -365,235 +366,239 @@ Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#53] Join condition: None -(59) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, ws_sold_date_sk#62] +(59) Project [codegen id : 24] +Output [8]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, CASE WHEN (store_sales#15 > 0.00) THEN CheckOverflow((promote_precision(store_sales#29) / promote_precision(store_sales#15)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#60] +Input [7]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58] + +(60) Scan parquet default.web_sales +Output [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#19)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(60) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, ws_sold_date_sk#62] +(61) ColumnarToRow [codegen id : 18] +Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] -(61) Filter [codegen id : 18] -Input [3]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, ws_sold_date_sk#62] -Condition : isnotnull(ws_bill_addr_sk#60) +(62) Filter [codegen id : 18] +Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_addr_sk#61) -(62) ReusedExchange [Reuses operator id: 98] -Output [3]: [d_date_sk#63, d_year#64, d_qoy#65] +(63) ReusedExchange [Reuses operator id: 99] +Output [3]: [d_date_sk#64, d_year#65, d_qoy#66] -(63) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#62] -Right keys [1]: [d_date_sk#63] +(64) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_sold_date_sk#63] +Right keys [1]: [d_date_sk#64] Join condition: None -(64) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, d_year#64, d_qoy#65] -Input [6]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, ws_sold_date_sk#62, d_date_sk#63, d_year#64, d_qoy#65] +(65) Project [codegen id : 18] +Output [4]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66] +Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65, d_qoy#66] -(65) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#66, ca_county#67] +(66) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#67, ca_county#68] -(66) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#60] -Right keys [1]: [ca_address_sk#66] +(67) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_bill_addr_sk#61] +Right keys [1]: [ca_address_sk#67] Join condition: None -(67) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#61, d_year#64, d_qoy#65, ca_county#67] -Input [6]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, d_year#64, d_qoy#65, ca_address_sk#66, ca_county#67] - -(68) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#61, d_year#64, d_qoy#65, ca_county#67] -Keys [3]: [ca_county#67, d_qoy#65, d_year#64] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#61))] -Aggregate Attributes [1]: [sum#68] -Results [4]: [ca_county#67, d_qoy#65, d_year#64, sum#69] - -(69) Exchange -Input [4]: [ca_county#67, d_qoy#65, d_year#64, sum#69] -Arguments: hashpartitioning(ca_county#67, d_qoy#65, d_year#64, 5), ENSURE_REQUIREMENTS, [id=#70] - -(70) HashAggregate [codegen id : 19] -Input [4]: [ca_county#67, d_qoy#65, d_year#64, sum#69] -Keys [3]: [ca_county#67, d_qoy#65, d_year#64] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#61))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#61))#71] -Results [2]: [ca_county#67, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#61))#71,17,2) AS web_sales#72] - -(71) BroadcastExchange -Input [2]: [ca_county#67, web_sales#72] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#73] - -(72) BroadcastHashJoin [codegen id : 24] +(68) Project [codegen id : 18] +Output [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] +Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_address_sk#67, ca_county#68] + +(69) HashAggregate [codegen id : 18] +Input [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] +Keys [3]: [ca_county#68, d_qoy#66, d_year#65] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#62))] +Aggregate Attributes [1]: [sum#69] +Results [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] + +(70) Exchange +Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] +Arguments: hashpartitioning(ca_county#68, d_qoy#66, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#71] + +(71) HashAggregate [codegen id : 19] +Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] +Keys [3]: [ca_county#68, d_qoy#66, d_year#65] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#62))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#62))#72] +Results [2]: [ca_county#68, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#62))#72,17,2) AS web_sales#73] + +(72) BroadcastExchange +Input [2]: [ca_county#68, web_sales#73] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#74] + +(73) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#53] -Right keys [1]: [ca_county#67] -Join condition: (CASE WHEN (web_sales#58 > 0.00) THEN CheckOverflow((promote_precision(web_sales#72) / promote_precision(web_sales#58)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#15 > 0.00) THEN CheckOverflow((promote_precision(store_sales#29) / promote_precision(store_sales#15)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [ca_county#68] +Join condition: (CASE WHEN (web_sales#58 > 0.00) THEN CheckOverflow((promote_precision(web_sales#73) / promote_precision(web_sales#58)), DecimalType(37,20), true) ELSE null END > CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#60) -(73) Project [codegen id : 24] -Output [8]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, web_sales#72] -Input [9]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, ca_county#67, web_sales#72] +(74) Project [codegen id : 24] +Output [9]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, web_sales#73, CASE WHEN (store_sales#29 > 0.00) THEN CheckOverflow((promote_precision(store_sales#44) / promote_precision(store_sales#29)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#75] +Input [10]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#60, ca_county#68, web_sales#73] -(74) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, ws_sold_date_sk#76] +(75) Scan parquet default.web_sales +Output [3]: [ws_bill_addr_sk#76, ws_ext_sales_price#77, ws_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#76), dynamicpruningexpression(ws_sold_date_sk#76 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_sold_date_sk#78 IN dynamicpruning#34)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(75) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, ws_sold_date_sk#76] +(76) ColumnarToRow [codegen id : 22] +Input [3]: [ws_bill_addr_sk#76, ws_ext_sales_price#77, ws_sold_date_sk#78] -(76) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, ws_sold_date_sk#76] -Condition : isnotnull(ws_bill_addr_sk#74) +(77) Filter [codegen id : 22] +Input [3]: [ws_bill_addr_sk#76, ws_ext_sales_price#77, ws_sold_date_sk#78] +Condition : isnotnull(ws_bill_addr_sk#76) -(77) ReusedExchange [Reuses operator id: 102] -Output [3]: [d_date_sk#77, d_year#78, d_qoy#79] +(78) ReusedExchange [Reuses operator id: 103] +Output [3]: [d_date_sk#79, d_year#80, d_qoy#81] -(78) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#76] -Right keys [1]: [d_date_sk#77] +(79) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_sold_date_sk#78] +Right keys [1]: [d_date_sk#79] Join condition: None -(79) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, d_year#78, d_qoy#79] -Input [6]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, ws_sold_date_sk#76, d_date_sk#77, d_year#78, d_qoy#79] +(80) Project [codegen id : 22] +Output [4]: [ws_bill_addr_sk#76, ws_ext_sales_price#77, d_year#80, d_qoy#81] +Input [6]: [ws_bill_addr_sk#76, ws_ext_sales_price#77, ws_sold_date_sk#78, d_date_sk#79, d_year#80, d_qoy#81] -(80) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#80, ca_county#81] +(81) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#82, ca_county#83] -(81) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#74] -Right keys [1]: [ca_address_sk#80] +(82) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_bill_addr_sk#76] +Right keys [1]: [ca_address_sk#82] Join condition: None -(82) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#75, d_year#78, d_qoy#79, ca_county#81] -Input [6]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, d_year#78, d_qoy#79, ca_address_sk#80, ca_county#81] - -(83) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#75, d_year#78, d_qoy#79, ca_county#81] -Keys [3]: [ca_county#81, d_qoy#79, d_year#78] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#75))] -Aggregate Attributes [1]: [sum#82] -Results [4]: [ca_county#81, d_qoy#79, d_year#78, sum#83] - -(84) Exchange -Input [4]: [ca_county#81, d_qoy#79, d_year#78, sum#83] -Arguments: hashpartitioning(ca_county#81, d_qoy#79, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#84] - -(85) HashAggregate [codegen id : 23] -Input [4]: [ca_county#81, d_qoy#79, d_year#78, sum#83] -Keys [3]: [ca_county#81, d_qoy#79, d_year#78] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#75))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#75))#85] -Results [2]: [ca_county#81, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#75))#85,17,2) AS web_sales#86] - -(86) BroadcastExchange -Input [2]: [ca_county#81, web_sales#86] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#87] - -(87) BroadcastHashJoin [codegen id : 24] +(83) Project [codegen id : 22] +Output [4]: [ws_ext_sales_price#77, d_year#80, d_qoy#81, ca_county#83] +Input [6]: [ws_bill_addr_sk#76, ws_ext_sales_price#77, d_year#80, d_qoy#81, ca_address_sk#82, ca_county#83] + +(84) HashAggregate [codegen id : 22] +Input [4]: [ws_ext_sales_price#77, d_year#80, d_qoy#81, ca_county#83] +Keys [3]: [ca_county#83, d_qoy#81, d_year#80] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#77))] +Aggregate Attributes [1]: [sum#84] +Results [4]: [ca_county#83, d_qoy#81, d_year#80, sum#85] + +(85) Exchange +Input [4]: [ca_county#83, d_qoy#81, d_year#80, sum#85] +Arguments: hashpartitioning(ca_county#83, d_qoy#81, d_year#80, 5), ENSURE_REQUIREMENTS, [id=#86] + +(86) HashAggregate [codegen id : 23] +Input [4]: [ca_county#83, d_qoy#81, d_year#80, sum#85] +Keys [3]: [ca_county#83, d_qoy#81, d_year#80] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#77))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#77))#87] +Results [2]: [ca_county#83, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#77))#87,17,2) AS web_sales#88] + +(87) BroadcastExchange +Input [2]: [ca_county#83, web_sales#88] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#89] + +(88) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#53] -Right keys [1]: [ca_county#81] -Join condition: (CASE WHEN (web_sales#72 > 0.00) THEN CheckOverflow((promote_precision(web_sales#86) / promote_precision(web_sales#72)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#29 > 0.00) THEN CheckOverflow((promote_precision(store_sales#44) / promote_precision(store_sales#29)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [ca_county#83] +Join condition: (CASE WHEN (web_sales#73 > 0.00) THEN CheckOverflow((promote_precision(web_sales#88) / promote_precision(web_sales#73)), DecimalType(37,20), true) ELSE null END > CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#75) -(88) Project [codegen id : 24] -Output [6]: [ca_county#9, d_year#6, CheckOverflow((promote_precision(web_sales#72) / promote_precision(web_sales#58)), DecimalType(37,20), true) AS web_q1_q2_increase#88, CheckOverflow((promote_precision(store_sales#29) / promote_precision(store_sales#15)), DecimalType(37,20), true) AS store_q1_q2_increase#89, CheckOverflow((promote_precision(web_sales#86) / promote_precision(web_sales#72)), DecimalType(37,20), true) AS web_q2_q3_increase#90, CheckOverflow((promote_precision(store_sales#44) / promote_precision(store_sales#29)), DecimalType(37,20), true) AS store_q2_q3_increase#91] -Input [10]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, web_sales#72, ca_county#81, web_sales#86] +(89) Project [codegen id : 24] +Output [6]: [ca_county#9, d_year#6, CheckOverflow((promote_precision(web_sales#73) / promote_precision(web_sales#58)), DecimalType(37,20), true) AS web_q1_q2_increase#90, CheckOverflow((promote_precision(store_sales#29) / promote_precision(store_sales#15)), DecimalType(37,20), true) AS store_q1_q2_increase#91, CheckOverflow((promote_precision(web_sales#88) / promote_precision(web_sales#73)), DecimalType(37,20), true) AS web_q2_q3_increase#92, CheckOverflow((promote_precision(store_sales#44) / promote_precision(store_sales#29)), DecimalType(37,20), true) AS store_q2_q3_increase#93] +Input [11]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, web_sales#73, CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#75, ca_county#83, web_sales#88] -(89) Exchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#88, store_q1_q2_increase#89, web_q2_q3_increase#90, store_q2_q3_increase#91] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#92] +(90) Exchange +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#90, store_q1_q2_increase#91, web_q2_q3_increase#92, store_q2_q3_increase#93] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#94] -(90) Sort [codegen id : 25] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#88, store_q1_q2_increase#89, web_q2_q3_increase#90, store_q2_q3_increase#91] +(91) Sort [codegen id : 25] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#90, store_q1_q2_increase#91, web_q2_q3_increase#92, store_q2_q3_increase#93] Arguments: [ca_county#9 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (94) -+- * Filter (93) - +- * ColumnarToRow (92) - +- Scan parquet default.date_dim (91) +BroadcastExchange (95) ++- * Filter (94) + +- * ColumnarToRow (93) + +- Scan parquet default.date_dim (92) -(91) Scan parquet default.date_dim +(92) Scan parquet default.date_dim Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 1] +(93) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(93) Filter [codegen id : 1] +(94) Filter [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(94) BroadcastExchange +(95) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#93] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (98) -+- * Filter (97) - +- * ColumnarToRow (96) - +- Scan parquet default.date_dim (95) +BroadcastExchange (99) ++- * Filter (98) + +- * ColumnarToRow (97) + +- Scan parquet default.date_dim (96) -(95) Scan parquet default.date_dim +(96) Scan parquet default.date_dim Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(96) ColumnarToRow [codegen id : 1] +(97) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] -(97) Filter [codegen id : 1] +(98) Filter [codegen id : 1] Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] Condition : ((((isnotnull(d_qoy#22) AND isnotnull(d_year#21)) AND (d_qoy#22 = 2)) AND (d_year#21 = 2000)) AND isnotnull(d_date_sk#20)) -(98) BroadcastExchange +(99) BroadcastExchange Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#94] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#34 -BroadcastExchange (102) -+- * Filter (101) - +- * ColumnarToRow (100) - +- Scan parquet default.date_dim (99) +BroadcastExchange (103) ++- * Filter (102) + +- * ColumnarToRow (101) + +- Scan parquet default.date_dim (100) -(99) Scan parquet default.date_dim +(100) Scan parquet default.date_dim Output [3]: [d_date_sk#35, d_year#36, d_qoy#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 1] +(101) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#35, d_year#36, d_qoy#37] -(101) Filter [codegen id : 1] +(102) Filter [codegen id : 1] Input [3]: [d_date_sk#35, d_year#36, d_qoy#37] Condition : ((((isnotnull(d_qoy#37) AND isnotnull(d_year#36)) AND (d_qoy#37 = 3)) AND (d_year#36 = 2000)) AND isnotnull(d_date_sk#35)) -(102) BroadcastExchange +(103) BroadcastExchange Input [3]: [d_date_sk#35, d_year#36, d_qoy#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#97] Subquery:4 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 59 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#19 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#19 -Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#76 IN dynamicpruning#34 +Subquery:6 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#78 IN dynamicpruning#34 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index 8e57ed5108baf..21869f77f47cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -4,49 +4,77 @@ WholeStageCodegen (25) Exchange [ca_county] #1 WholeStageCodegen (24) Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END] Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END] + Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales] + BroadcastHashJoin [ca_county,ca_county] + Project [ca_county,d_year,store_sales,store_sales,store_sales] BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (3) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [d_qoy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [ca_address_sk,ca_county] + BroadcastHashJoin [ca_county,ca_county] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #2 + WholeStageCodegen (3) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county] + Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [d_qoy,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [ca_address_sk,ca_county] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #6 + WholeStageCodegen (6) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + Filter [d_qoy,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #7 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) + BroadcastExchange #8 + WholeStageCodegen (11) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #6 - WholeStageCodegen (6) + Exchange [ca_county,d_qoy,d_year] #9 + WholeStageCodegen (10) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,d_year,d_qoy,ca_county] BroadcastHashJoin [ss_addr_sk,ca_address_sk] @@ -56,65 +84,38 @@ WholeStageCodegen (25) ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #10 WholeStageCodegen (1) Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 + ReusedExchange [d_date_sk,d_year,d_qoy] #10 InputAdapter ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #9 - WholeStageCodegen (10) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #10 - WholeStageCodegen (1) - Filter [d_qoy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #12 - WholeStageCodegen (14) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #12 + WholeStageCodegen (14) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 InputAdapter BroadcastExchange #13 WholeStageCodegen (19) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt index 1ace9e7f294aa..70c3d52114c27 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -CollectLimit (27) -+- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildLeft (22) +CollectLimit (28) ++- * Project (27) + +- * BroadcastHashJoin Inner BuildRight (26) + :- * Project (24) + : +- * BroadcastHashJoin Inner BuildLeft (23) : :- BroadcastExchange (18) : : +- * Project (17) : : +- * BroadcastHashJoin Inner BuildLeft (16) @@ -22,10 +22,11 @@ CollectLimit (27) : : : +- * ColumnarToRow (7) : : : +- Scan parquet default.catalog_sales (6) : : +- ReusedExchange (9) - : +- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet default.catalog_sales (19) - +- ReusedExchange (24) + : +- * Project (22) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.catalog_sales (19) + +- ReusedExchange (25) (1) Scan parquet default.item @@ -65,7 +66,7 @@ Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] Condition : isnotnull(cs_item_sk#4) -(9) ReusedExchange [Reuses operator id: 32] +(9) ReusedExchange [Reuses operator id: 33] Output [1]: [d_date_sk#8] (10) BroadcastHashJoin [codegen id : 3] @@ -127,62 +128,66 @@ Input [3]: [cs_item_sk#17, cs_ext_discount_amt#18, cs_sold_date_sk#19] Input [3]: [cs_item_sk#17, cs_ext_discount_amt#18, cs_sold_date_sk#19] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_ext_discount_amt#18)) -(22) BroadcastHashJoin [codegen id : 6] +(22) Project +Output [3]: [cs_item_sk#17, cs_sold_date_sk#19, cast(cs_ext_discount_amt#18 as decimal(14,7)) AS CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7))#20] +Input [3]: [cs_item_sk#17, cs_ext_discount_amt#18, cs_sold_date_sk#19] + +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] Right keys [1]: [cs_item_sk#17] -Join condition: (cast(cs_ext_discount_amt#18 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#15) +Join condition: (CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7))#20 > (1.3 * avg(cs_ext_discount_amt))#15) -(23) Project [codegen id : 6] +(24) Project [codegen id : 6] Output [1]: [cs_sold_date_sk#19] -Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#17, cs_ext_discount_amt#18, cs_sold_date_sk#19] +Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#17, cs_sold_date_sk#19, CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7))#20] -(24) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#20] +(25) ReusedExchange [Reuses operator id: 33] +Output [1]: [d_date_sk#21] -(25) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None -(26) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #21] -Input [2]: [cs_sold_date_sk#19, d_date_sk#20] +(27) Project [codegen id : 6] +Output [1]: [1 AS excess discount amount #22] +Input [2]: [cs_sold_date_sk#19, d_date_sk#21] -(27) CollectLimit -Input [1]: [excess discount amount #21] +(28) CollectLimit +Input [1]: [excess discount amount #22] Arguments: 100 ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = cs_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (32) -+- * Project (31) - +- * Filter (30) - +- * ColumnarToRow (29) - +- Scan parquet default.date_dim (28) +BroadcastExchange (33) ++- * Project (32) + +- * Filter (31) + +- * ColumnarToRow (30) + +- Scan parquet default.date_dim (29) -(28) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#22] +(29) Scan parquet default.date_dim +Output [2]: [d_date_sk#8, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#8, d_date#22] +(30) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#8, d_date#23] -(30) Filter [codegen id : 1] -Input [2]: [d_date_sk#8, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#22 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) +(31) Filter [codegen id : 1] +Input [2]: [d_date_sk#8, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) -(31) Project [codegen id : 1] +(32) Project [codegen id : 1] Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#22] +Input [2]: [d_date_sk#8, d_date#23] -(32) BroadcastExchange +(33) BroadcastExchange Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] Subquery:2 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt index 8ca9bf49029f8..573c1fdc50c4d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt @@ -3,7 +3,7 @@ CollectLimit Project BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] + BroadcastHashJoin [i_item_sk,cs_item_sk,CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7)),(1.3 * avg(cs_ext_discount_amt))] InputAdapter BroadcastExchange #1 WholeStageCodegen (4) @@ -39,10 +39,11 @@ CollectLimit Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #4 - Filter [cs_item_sk,cs_ext_discount_amt] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Project [cs_item_sk,cs_sold_date_sk,cs_ext_discount_amt] + Filter [cs_item_sk,cs_ext_discount_amt] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index f6c9b9ed7dcef..0c0b8c71f5ae9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -71,85 +71,85 @@ Right keys [1]: [i_item_sk#5] Join condition: None (10) Project [codegen id : 6] -Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] +Output [3]: [cs_sold_date_sk#3, i_item_sk#5, cast(cs_ext_discount_amt#2 as decimal(14,7)) AS CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7))#8] Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] +Output [3]: [cs_item_sk#9, cs_ext_discount_amt#10, cs_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#4)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] +Input [3]: [cs_item_sk#9, cs_ext_discount_amt#10, cs_sold_date_sk#11] (13) Filter [codegen id : 3] -Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] -Condition : isnotnull(cs_item_sk#8) +Input [3]: [cs_item_sk#9, cs_ext_discount_amt#10, cs_sold_date_sk#11] +Condition : isnotnull(cs_item_sk#9) (14) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#12] (15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#10] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [cs_sold_date_sk#11] +Right keys [1]: [d_date_sk#12] Join condition: None (16) Project [codegen id : 3] -Output [2]: [cs_item_sk#8, cs_ext_discount_amt#9] -Input [4]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10, d_date_sk#11] +Output [2]: [cs_item_sk#9, cs_ext_discount_amt#10] +Input [4]: [cs_item_sk#9, cs_ext_discount_amt#10, cs_sold_date_sk#11, d_date_sk#12] (17) HashAggregate [codegen id : 3] -Input [2]: [cs_item_sk#8, cs_ext_discount_amt#9] -Keys [1]: [cs_item_sk#8] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#9))] -Aggregate Attributes [2]: [sum#12, count#13] -Results [3]: [cs_item_sk#8, sum#14, count#15] +Input [2]: [cs_item_sk#9, cs_ext_discount_amt#10] +Keys [1]: [cs_item_sk#9] +Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#10))] +Aggregate Attributes [2]: [sum#13, count#14] +Results [3]: [cs_item_sk#9, sum#15, count#16] (18) Exchange -Input [3]: [cs_item_sk#8, sum#14, count#15] -Arguments: hashpartitioning(cs_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [3]: [cs_item_sk#9, sum#15, count#16] +Arguments: hashpartitioning(cs_item_sk#9, 5), ENSURE_REQUIREMENTS, [id=#17] (19) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#8, sum#14, count#15] -Keys [1]: [cs_item_sk#8] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))#17] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#9))#17 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#18, cs_item_sk#8] +Input [3]: [cs_item_sk#9, sum#15, count#16] +Keys [1]: [cs_item_sk#9] +Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#10))] +Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#10))#18] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#10))#18 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#19, cs_item_sk#9] (20) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#18, cs_item_sk#8] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#18) +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#19, cs_item_sk#9] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#19) (21) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#18, cs_item_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#19, cs_item_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#20] (22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [cs_item_sk#8] -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#18) +Right keys [1]: [cs_item_sk#9] +Join condition: (CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7))#8 > (1.3 * avg(cs_ext_discount_amt))#19) (23) Project [codegen id : 6] Output [1]: [cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#18, cs_item_sk#8] +Input [5]: [cs_sold_date_sk#3, i_item_sk#5, CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7))#8, (1.3 * avg(cs_ext_discount_amt))#19, cs_item_sk#9] (24) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#21] (25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None (26) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #21] -Input [2]: [cs_sold_date_sk#3, d_date_sk#20] +Output [1]: [1 AS excess discount amount #22] +Input [2]: [cs_sold_date_sk#3, d_date_sk#21] (27) CollectLimit -Input [1]: [excess discount amount #21] +Input [1]: [excess discount amount #22] Arguments: 100 ===== Subqueries ===== @@ -163,27 +163,27 @@ BroadcastExchange (32) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#20, d_date#22] +Output [2]: [d_date_sk#21, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_date#22] +Input [2]: [d_date_sk#21, d_date#23] (30) Filter [codegen id : 1] -Input [2]: [d_date_sk#20, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#22 <= 2000-04-26)) AND isnotnull(d_date_sk#20)) +Input [2]: [d_date_sk#21, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#21)) (31) Project [codegen id : 1] -Output [1]: [d_date_sk#20] -Input [2]: [d_date_sk#20, d_date#22] +Output [1]: [d_date_sk#21] +Input [2]: [d_date_sk#21, d_date#23] (32) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [d_date_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index aea77be43cf05..fcc9dbc36ce95 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -3,8 +3,8 @@ CollectLimit Project BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] - Project [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk,CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7)),(1.3 * avg(cs_ext_discount_amt))] + Project [cs_sold_date_sk,i_item_sk,cs_ext_discount_amt] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_ext_discount_amt] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt index 4e4dd9555c718..107d5fb85a415 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt @@ -390,274 +390,274 @@ Right keys [1]: [customer_id#87] Join condition: None (60) Project [codegen id : 26] -Output [11]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#88] +Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#60) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#90] Input [12]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, customer_id#87, year_total#88] (61) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] +Output [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#95), dynamicpruningexpression(cs_sold_date_sk#95 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(cs_sold_date_sk#96), dynamicpruningexpression(cs_sold_date_sk#96 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 28] -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] +Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96] (63) Filter [codegen id : 28] -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Condition : isnotnull(cs_bill_customer_sk#90) +Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96] +Condition : isnotnull(cs_bill_customer_sk#91) (64) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#96, d_year#97] +Output [2]: [d_date_sk#97, d_year#98] (65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#95] -Right keys [1]: [d_date_sk#96] +Left keys [1]: [cs_sold_date_sk#96] +Right keys [1]: [d_date_sk#97] Join condition: None (66) Project [codegen id : 28] -Output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] -Input [8]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95, d_date_sk#96, d_year#97] +Output [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] +Input [8]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96, d_date_sk#97, d_year#98] (67) Exchange -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] -Arguments: hashpartitioning(cs_bill_customer_sk#90, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] +Arguments: hashpartitioning(cs_bill_customer_sk#91, 5), ENSURE_REQUIREMENTS, [id=#99] (68) Sort [codegen id : 29] -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] -Arguments: [cs_bill_customer_sk#90 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] +Arguments: [cs_bill_customer_sk#91 ASC NULLS FIRST], false, 0 (69) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] +Output [8]: [c_customer_sk#100, c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107] (70) Sort [codegen id : 31] -Input [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] -Arguments: [c_customer_sk#99 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#100, c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107] +Arguments: [c_customer_sk#100 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 32] -Left keys [1]: [cs_bill_customer_sk#90] -Right keys [1]: [c_customer_sk#99] +Left keys [1]: [cs_bill_customer_sk#91] +Right keys [1]: [c_customer_sk#100] Join condition: None (72) Project [codegen id : 32] -Output [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] -Input [14]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97, c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] +Output [12]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] +Input [14]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98, c_customer_sk#100, c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107] (73) HashAggregate [codegen id : 32] -Input [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] -Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#107, isEmpty#108] -Results [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, sum#109, isEmpty#110] +Input [12]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] +Keys [8]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#108, isEmpty#109] +Results [10]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, sum#110, isEmpty#111] (74) Exchange -Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, sum#109, isEmpty#110] -Arguments: hashpartitioning(c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, 5), ENSURE_REQUIREMENTS, [id=#111] +Input [10]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, sum#110, isEmpty#111] +Arguments: hashpartitioning(c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, 5), ENSURE_REQUIREMENTS, [id=#112] (75) HashAggregate [codegen id : 33] -Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, sum#109, isEmpty#110] -Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112] -Results [2]: [c_customer_id#100 AS customer_id#113, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112 AS year_total#114] +Input [10]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, sum#110, isEmpty#111] +Keys [8]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#113] +Results [2]: [c_customer_id#101 AS customer_id#114, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#113 AS year_total#115] (76) Exchange -Input [2]: [customer_id#113, year_total#114] -Arguments: hashpartitioning(customer_id#113, 5), ENSURE_REQUIREMENTS, [id=#115] +Input [2]: [customer_id#114, year_total#115] +Arguments: hashpartitioning(customer_id#114, 5), ENSURE_REQUIREMENTS, [id=#116] (77) Sort [codegen id : 34] -Input [2]: [customer_id#113, year_total#114] -Arguments: [customer_id#113 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#114, year_total#115] +Arguments: [customer_id#114 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#113] -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#60) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#114] +Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#115) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#90) (79) Project [codegen id : 35] -Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114] -Input [13]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#88, customer_id#113, year_total#114] +Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#115] +Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#90, customer_id#114, year_total#115] (80) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121] +Output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#122), dynamicpruningexpression(ws_sold_date_sk#122 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 37] -Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121] +Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] (82) Filter [codegen id : 37] -Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121] -Condition : isnotnull(ws_bill_customer_sk#116) +Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] +Condition : isnotnull(ws_bill_customer_sk#117) (83) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#122, d_year#123] +Output [2]: [d_date_sk#123, d_year#124] (84) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_sold_date_sk#121] -Right keys [1]: [d_date_sk#122] +Left keys [1]: [ws_sold_date_sk#122] +Right keys [1]: [d_date_sk#123] Join condition: None (85) Project [codegen id : 37] -Output [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] -Input [8]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121, d_date_sk#122, d_year#123] +Output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] +Input [8]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122, d_date_sk#123, d_year#124] (86) Exchange -Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] -Arguments: hashpartitioning(ws_bill_customer_sk#116, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] +Arguments: hashpartitioning(ws_bill_customer_sk#117, 5), ENSURE_REQUIREMENTS, [id=#125] (87) Sort [codegen id : 38] -Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] -Arguments: [ws_bill_customer_sk#116 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] +Arguments: [ws_bill_customer_sk#117 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +Output [8]: [c_customer_sk#126, c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133] (89) Sort [codegen id : 40] -Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] -Arguments: [c_customer_sk#125 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#126, c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133] +Arguments: [c_customer_sk#126 ASC NULLS FIRST], false, 0 (90) SortMergeJoin [codegen id : 41] -Left keys [1]: [ws_bill_customer_sk#116] -Right keys [1]: [c_customer_sk#125] +Left keys [1]: [ws_bill_customer_sk#117] +Right keys [1]: [c_customer_sk#126] Join condition: None (91) Project [codegen id : 41] -Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] -Input [14]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123, c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +Output [12]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] +Input [14]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124, c_customer_sk#126, c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133] (92) HashAggregate [codegen id : 41] -Input [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] -Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#133, isEmpty#134] -Results [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, sum#135, isEmpty#136] +Input [12]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] +Keys [8]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#134, isEmpty#135] +Results [10]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, sum#136, isEmpty#137] (93) Exchange -Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, sum#135, isEmpty#136] -Arguments: hashpartitioning(c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [10]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, sum#136, isEmpty#137] +Arguments: hashpartitioning(c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, 5), ENSURE_REQUIREMENTS, [id=#138] (94) HashAggregate [codegen id : 42] -Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, sum#135, isEmpty#136] -Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138] -Results [2]: [c_customer_id#126 AS customer_id#139, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138 AS year_total#140] +Input [10]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, sum#136, isEmpty#137] +Keys [8]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#139] +Results [2]: [c_customer_id#127 AS customer_id#140, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#139 AS year_total#141] (95) Filter [codegen id : 42] -Input [2]: [customer_id#139, year_total#140] -Condition : (isnotnull(year_total#140) AND (year_total#140 > 0.000000)) +Input [2]: [customer_id#140, year_total#141] +Condition : (isnotnull(year_total#141) AND (year_total#141 > 0.000000)) (96) Project [codegen id : 42] -Output [2]: [customer_id#139 AS customer_id#141, year_total#140 AS year_total#142] -Input [2]: [customer_id#139, year_total#140] +Output [2]: [customer_id#140 AS customer_id#142, year_total#141 AS year_total#143] +Input [2]: [customer_id#140, year_total#141] (97) Exchange -Input [2]: [customer_id#141, year_total#142] -Arguments: hashpartitioning(customer_id#141, 5), ENSURE_REQUIREMENTS, [id=#143] +Input [2]: [customer_id#142, year_total#143] +Arguments: hashpartitioning(customer_id#142, 5), ENSURE_REQUIREMENTS, [id=#144] (98) Sort [codegen id : 43] -Input [2]: [customer_id#141, year_total#142] -Arguments: [customer_id#141 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#142, year_total#143] +Arguments: [customer_id#142 ASC NULLS FIRST], false, 0 (99) SortMergeJoin [codegen id : 44] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#141] +Right keys [1]: [customer_id#142] Join condition: None (100) Project [codegen id : 44] -Output [11]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114, year_total#142] -Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114, customer_id#141, year_total#142] +Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#143, CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#115) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END AS CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#145] +Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#115, customer_id#142, year_total#143] (101) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] +Output [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#149), dynamicpruningexpression(ws_sold_date_sk#149 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(ws_sold_date_sk#151), dynamicpruningexpression(ws_sold_date_sk#151 IN dynamicpruning#35)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 46] -Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] +Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] (103) Filter [codegen id : 46] -Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] -Condition : isnotnull(ws_bill_customer_sk#144) +Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] +Condition : isnotnull(ws_bill_customer_sk#146) (104) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#150, d_year#151] +Output [2]: [d_date_sk#152, d_year#153] (105) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [ws_sold_date_sk#149] -Right keys [1]: [d_date_sk#150] +Left keys [1]: [ws_sold_date_sk#151] +Right keys [1]: [d_date_sk#152] Join condition: None (106) Project [codegen id : 46] -Output [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] -Input [8]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149, d_date_sk#150, d_year#151] +Output [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Input [8]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151, d_date_sk#152, d_year#153] (107) Exchange -Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] -Arguments: hashpartitioning(ws_bill_customer_sk#144, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Arguments: hashpartitioning(ws_bill_customer_sk#146, 5), ENSURE_REQUIREMENTS, [id=#154] (108) Sort [codegen id : 47] -Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] -Arguments: [ws_bill_customer_sk#144 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Arguments: [ws_bill_customer_sk#146 ASC NULLS FIRST], false, 0 (109) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#153, c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160] +Output [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] (110) Sort [codegen id : 49] -Input [8]: [c_customer_sk#153, c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160] -Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] +Arguments: [c_customer_sk#155 ASC NULLS FIRST], false, 0 (111) SortMergeJoin [codegen id : 50] -Left keys [1]: [ws_bill_customer_sk#144] -Right keys [1]: [c_customer_sk#153] +Left keys [1]: [ws_bill_customer_sk#146] +Right keys [1]: [c_customer_sk#155] Join condition: None (112) Project [codegen id : 50] -Output [12]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] -Input [14]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151, c_customer_sk#153, c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160] +Output [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Input [14]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153, c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] (113) HashAggregate [codegen id : 50] -Input [12]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] -Keys [8]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#161, isEmpty#162] -Results [10]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, sum#163, isEmpty#164] +Input [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#163, isEmpty#164] +Results [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] (114) Exchange -Input [10]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, sum#163, isEmpty#164] -Arguments: hashpartitioning(c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, 5), ENSURE_REQUIREMENTS, [id=#165] +Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Arguments: hashpartitioning(c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, 5), ENSURE_REQUIREMENTS, [id=#167] (115) HashAggregate [codegen id : 51] -Input [10]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, sum#163, isEmpty#164] -Keys [8]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166] -Results [2]: [c_customer_id#154 AS customer_id#167, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166 AS year_total#168] +Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#168] +Results [2]: [c_customer_id#156 AS customer_id#169, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#168 AS year_total#170] (116) Exchange -Input [2]: [customer_id#167, year_total#168] -Arguments: hashpartitioning(customer_id#167, 5), ENSURE_REQUIREMENTS, [id=#169] +Input [2]: [customer_id#169, year_total#170] +Arguments: hashpartitioning(customer_id#169, 5), ENSURE_REQUIREMENTS, [id=#171] (117) Sort [codegen id : 52] -Input [2]: [customer_id#167, year_total#168] -Arguments: [customer_id#167 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#169, year_total#170] +Arguments: [customer_id#169 ASC NULLS FIRST], false, 0 (118) SortMergeJoin [codegen id : 53] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#167] -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#142 > 0.000000) THEN CheckOverflow((promote_precision(year_total#168) / promote_precision(year_total#142)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#169] +Join condition: (CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#145 > CASE WHEN (year_total#143 > 0.000000) THEN CheckOverflow((promote_precision(year_total#170) / promote_precision(year_total#143)), DecimalType(38,14), true) ELSE null END) (119) Project [codegen id : 53] Output [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] -Input [13]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114, year_total#142, customer_id#167, year_total#168] +Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#143, CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#145, customer_id#169, year_total#170] (120) TakeOrderedAndProject Input [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] @@ -688,7 +688,7 @@ Condition : ((isnotnull(d_year#9) AND (d_year#9 = 2001)) AND isnotnull(d_date_sk (124) BroadcastExchange Input [2]: [d_date_sk#8, d_year#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#170] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#172] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#34 IN dynamicpruning#35 BroadcastExchange (128) @@ -713,14 +713,14 @@ Condition : ((isnotnull(d_year#37) AND (d_year#37 = 2002)) AND isnotnull(d_date_ (128) BroadcastExchange Input [2]: [d_date_sk#36, d_year#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#171] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#173] Subquery:3 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#7 -Subquery:4 Hosting operator id = 61 Hosting Expression = cs_sold_date_sk#95 IN dynamicpruning#35 +Subquery:4 Hosting operator id = 61 Hosting Expression = cs_sold_date_sk#96 IN dynamicpruning#35 -Subquery:5 Hosting operator id = 80 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#7 +Subquery:5 Hosting operator id = 80 Hosting Expression = ws_sold_date_sk#122 IN dynamicpruning#7 -Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#149 IN dynamicpruning#35 +Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#151 IN dynamicpruning#35 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt index d7e0a660bab5b..eab288e1d9891 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] WholeStageCodegen (53) Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + SortMergeJoin [customer_id,customer_id,CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END,year_total,year_total] InputAdapter WholeStageCodegen (44) Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] @@ -9,10 +9,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter WholeStageCodegen (35) Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END] InputAdapter WholeStageCodegen (26) - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] + Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt index d6b8e5110ca95..c2c3f165c3940 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt @@ -358,271 +358,271 @@ Right keys [1]: [customer_id#85] Join condition: None (53) Project [codegen id : 24] -Output [11]: [customer_id#25, year_total#26, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#58, year_total#86] +Output [10]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, CASE WHEN (year_total#26 > 0.000000) THEN CheckOverflow((promote_precision(year_total#58) / promote_precision(year_total#26)), DecimalType(38,14), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#88] Input [12]: [customer_id#25, year_total#26, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#58, customer_id#85, year_total#86] (54) Scan parquet default.customer -Output [8]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95] +Output [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (55) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95] +Input [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] (56) Filter [codegen id : 14] -Input [8]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95] -Condition : (isnotnull(c_customer_sk#88) AND isnotnull(c_customer_id#89)) +Input [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] +Condition : (isnotnull(c_customer_sk#89) AND isnotnull(c_customer_id#90)) (57) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#96, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101] +Output [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#101), dynamicpruningexpression(cs_sold_date_sk#101 IN dynamicpruning#41)] +PartitionFilters: [isnotnull(cs_sold_date_sk#102), dynamicpruningexpression(cs_sold_date_sk#102 IN dynamicpruning#41)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#96, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101] +Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] (59) Filter [codegen id : 12] -Input [6]: [cs_bill_customer_sk#96, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101] -Condition : isnotnull(cs_bill_customer_sk#96) +Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] +Condition : isnotnull(cs_bill_customer_sk#97) (60) BroadcastExchange -Input [6]: [cs_bill_customer_sk#96, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#102] +Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] (61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#88] -Right keys [1]: [cs_bill_customer_sk#96] +Left keys [1]: [c_customer_sk#89] +Right keys [1]: [cs_bill_customer_sk#97] Join condition: None (62) Project [codegen id : 14] -Output [12]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101] -Input [14]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, cs_bill_customer_sk#96, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101] +Output [12]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] +Input [14]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] (63) ReusedExchange [Reuses operator id: 118] -Output [2]: [d_date_sk#103, d_year#104] +Output [2]: [d_date_sk#104, d_year#105] (64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#101] -Right keys [1]: [d_date_sk#103] +Left keys [1]: [cs_sold_date_sk#102] +Right keys [1]: [d_date_sk#104] Join condition: None (65) Project [codegen id : 14] -Output [12]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, d_year#104] -Input [14]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101, d_date_sk#103, d_year#104] +Output [12]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, d_year#105] +Input [14]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102, d_date_sk#104, d_year#105] (66) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, d_year#104] -Keys [8]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, d_year#104] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#100 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#99 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#97 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#98 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#105, isEmpty#106] -Results [10]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, d_year#104, sum#107, isEmpty#108] +Input [12]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, d_year#105] +Keys [8]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, d_year#105] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#101 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#100 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#98 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#99 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#106, isEmpty#107] +Results [10]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, d_year#105, sum#108, isEmpty#109] (67) Exchange -Input [10]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, d_year#104, sum#107, isEmpty#108] -Arguments: hashpartitioning(c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, d_year#104, 5), ENSURE_REQUIREMENTS, [id=#109] +Input [10]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, d_year#105, sum#108, isEmpty#109] +Arguments: hashpartitioning(c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, d_year#105, 5), ENSURE_REQUIREMENTS, [id=#110] (68) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, d_year#104, sum#107, isEmpty#108] -Keys [8]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, d_year#104] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#100 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#99 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#97 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#98 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#100 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#99 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#97 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#98 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#110] -Results [2]: [c_customer_id#89 AS customer_id#111, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#100 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#99 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#97 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#98 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#110 AS year_total#112] +Input [10]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, d_year#105, sum#108, isEmpty#109] +Keys [8]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, d_year#105] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#101 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#100 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#98 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#99 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#101 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#100 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#98 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#99 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#111] +Results [2]: [c_customer_id#90 AS customer_id#112, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#101 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#100 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#98 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#99 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#111 AS year_total#113] (69) BroadcastExchange -Input [2]: [customer_id#111, year_total#112] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#113] +Input [2]: [customer_id#112, year_total#113] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#114] (70) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#111] -Join condition: (CASE WHEN (year_total#86 > 0.000000) THEN CheckOverflow((promote_precision(year_total#112) / promote_precision(year_total#86)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#26 > 0.000000) THEN CheckOverflow((promote_precision(year_total#58) / promote_precision(year_total#26)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#112] +Join condition: (CASE WHEN (year_total#86 > 0.000000) THEN CheckOverflow((promote_precision(year_total#113) / promote_precision(year_total#86)), DecimalType(38,14), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#88) (71) Project [codegen id : 24] -Output [10]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, year_total#112] -Input [13]: [customer_id#25, year_total#26, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#58, year_total#86, customer_id#111, year_total#112] +Output [10]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, year_total#113] +Input [12]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#88, customer_id#112, year_total#113] (72) Scan parquet default.customer -Output [8]: [c_customer_sk#114, c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121] +Output [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (73) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#114, c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121] +Input [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] (74) Filter [codegen id : 18] -Input [8]: [c_customer_sk#114, c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121] -Condition : (isnotnull(c_customer_sk#114) AND isnotnull(c_customer_id#115)) +Input [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] +Condition : (isnotnull(c_customer_sk#115) AND isnotnull(c_customer_id#116)) (75) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] +Output [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#127), dynamicpruningexpression(ws_sold_date_sk#127 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(ws_sold_date_sk#128), dynamicpruningexpression(ws_sold_date_sk#128 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] +Input [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] (77) Filter [codegen id : 16] -Input [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] -Condition : isnotnull(ws_bill_customer_sk#122) +Input [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] +Condition : isnotnull(ws_bill_customer_sk#123) (78) BroadcastExchange -Input [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#128] +Input [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#129] (79) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#114] -Right keys [1]: [ws_bill_customer_sk#122] +Left keys [1]: [c_customer_sk#115] +Right keys [1]: [ws_bill_customer_sk#123] Join condition: None (80) Project [codegen id : 18] -Output [12]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] -Input [14]: [c_customer_sk#114, c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] +Output [12]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] +Input [14]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] (81) ReusedExchange [Reuses operator id: 114] -Output [2]: [d_date_sk#129, d_year#130] +Output [2]: [d_date_sk#130, d_year#131] (82) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#127] -Right keys [1]: [d_date_sk#129] +Left keys [1]: [ws_sold_date_sk#128] +Right keys [1]: [d_date_sk#130] Join condition: None (83) Project [codegen id : 18] -Output [12]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, d_year#130] -Input [14]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127, d_date_sk#129, d_year#130] +Output [12]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, d_year#131] +Input [14]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128, d_date_sk#130, d_year#131] (84) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, d_year#130] -Keys [8]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, d_year#130] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#126 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#125 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#123 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#124 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#131, isEmpty#132] -Results [10]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, d_year#130, sum#133, isEmpty#134] +Input [12]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, d_year#131] +Keys [8]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#127 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#126 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#124 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#125 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#132, isEmpty#133] +Results [10]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, sum#134, isEmpty#135] (85) Exchange -Input [10]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, d_year#130, sum#133, isEmpty#134] -Arguments: hashpartitioning(c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, d_year#130, 5), ENSURE_REQUIREMENTS, [id=#135] +Input [10]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, sum#134, isEmpty#135] +Arguments: hashpartitioning(c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, 5), ENSURE_REQUIREMENTS, [id=#136] (86) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, d_year#130, sum#133, isEmpty#134] -Keys [8]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, d_year#130] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#126 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#125 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#123 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#124 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#126 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#125 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#123 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#124 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#136] -Results [2]: [c_customer_id#115 AS customer_id#137, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#126 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#125 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#123 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#124 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#136 AS year_total#138] +Input [10]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, sum#134, isEmpty#135] +Keys [8]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#127 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#126 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#124 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#125 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#127 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#126 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#124 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#125 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#137] +Results [2]: [c_customer_id#116 AS customer_id#138, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#127 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#126 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#124 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#125 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#137 AS year_total#139] (87) Filter [codegen id : 19] -Input [2]: [customer_id#137, year_total#138] -Condition : (isnotnull(year_total#138) AND (year_total#138 > 0.000000)) +Input [2]: [customer_id#138, year_total#139] +Condition : (isnotnull(year_total#139) AND (year_total#139 > 0.000000)) (88) Project [codegen id : 19] -Output [2]: [customer_id#137 AS customer_id#139, year_total#138 AS year_total#140] -Input [2]: [customer_id#137, year_total#138] +Output [2]: [customer_id#138 AS customer_id#140, year_total#139 AS year_total#141] +Input [2]: [customer_id#138, year_total#139] (89) BroadcastExchange -Input [2]: [customer_id#139, year_total#140] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#141] +Input [2]: [customer_id#140, year_total#141] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#142] (90) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#139] +Right keys [1]: [customer_id#140] Join condition: None (91) Project [codegen id : 24] -Output [11]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, year_total#112, year_total#140] -Input [12]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, year_total#112, customer_id#139, year_total#140] +Output [10]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#141, CASE WHEN (year_total#86 > 0.000000) THEN CheckOverflow((promote_precision(year_total#113) / promote_precision(year_total#86)), DecimalType(38,14), true) ELSE null END AS CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#143] +Input [12]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, year_total#113, customer_id#140, year_total#141] (92) Scan parquet default.customer -Output [8]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149] +Output [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (93) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149] +Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] (94) Filter [codegen id : 22] -Input [8]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149] -Condition : (isnotnull(c_customer_sk#142) AND isnotnull(c_customer_id#143)) +Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] +Condition : (isnotnull(c_customer_sk#144) AND isnotnull(c_customer_id#145)) (95) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#150, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155] +Output [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#155), dynamicpruningexpression(ws_sold_date_sk#155 IN dynamicpruning#41)] +PartitionFilters: [isnotnull(ws_sold_date_sk#157), dynamicpruningexpression(ws_sold_date_sk#157 IN dynamicpruning#41)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (96) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#150, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155] +Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] (97) Filter [codegen id : 20] -Input [6]: [ws_bill_customer_sk#150, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155] -Condition : isnotnull(ws_bill_customer_sk#150) +Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Condition : isnotnull(ws_bill_customer_sk#152) (98) BroadcastExchange -Input [6]: [ws_bill_customer_sk#150, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#156] +Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#158] (99) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#142] -Right keys [1]: [ws_bill_customer_sk#150] +Left keys [1]: [c_customer_sk#144] +Right keys [1]: [ws_bill_customer_sk#152] Join condition: None (100) Project [codegen id : 22] -Output [12]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155] -Input [14]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, ws_bill_customer_sk#150, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155] +Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Input [14]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] (101) ReusedExchange [Reuses operator id: 118] -Output [2]: [d_date_sk#157, d_year#158] +Output [2]: [d_date_sk#159, d_year#160] (102) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#155] -Right keys [1]: [d_date_sk#157] +Left keys [1]: [ws_sold_date_sk#157] +Right keys [1]: [d_date_sk#159] Join condition: None (103) Project [codegen id : 22] -Output [12]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, d_year#158] -Input [14]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155, d_date_sk#157, d_year#158] +Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] +Input [14]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157, d_date_sk#159, d_year#160] (104) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, d_year#158] -Keys [8]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, d_year#158] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#154 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#153 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#151 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#152 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#159, isEmpty#160] -Results [10]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, d_year#158, sum#161, isEmpty#162] +Input [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] +Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#161, isEmpty#162] +Results [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] (105) Exchange -Input [10]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, d_year#158, sum#161, isEmpty#162] -Arguments: hashpartitioning(c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, d_year#158, 5), ENSURE_REQUIREMENTS, [id=#163] +Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] +Arguments: hashpartitioning(c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, 5), ENSURE_REQUIREMENTS, [id=#165] (106) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, d_year#158, sum#161, isEmpty#162] -Keys [8]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, d_year#158] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#154 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#153 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#151 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#152 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#154 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#153 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#151 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#152 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#164] -Results [2]: [c_customer_id#143 AS customer_id#165, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#154 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#153 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#151 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#152 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#164 AS year_total#166] +Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] +Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166] +Results [2]: [c_customer_id#145 AS customer_id#167, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166 AS year_total#168] (107) BroadcastExchange -Input [2]: [customer_id#165, year_total#166] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#167] +Input [2]: [customer_id#167, year_total#168] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#169] (108) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#165] -Join condition: (CASE WHEN (year_total#86 > 0.000000) THEN CheckOverflow((promote_precision(year_total#112) / promote_precision(year_total#86)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#140 > 0.000000) THEN CheckOverflow((promote_precision(year_total#166) / promote_precision(year_total#140)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#167] +Join condition: (CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#143 > CASE WHEN (year_total#141 > 0.000000) THEN CheckOverflow((promote_precision(year_total#168) / promote_precision(year_total#141)), DecimalType(38,14), true) ELSE null END) (109) Project [codegen id : 24] Output [7]: [customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57] -Input [13]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, year_total#112, year_total#140, customer_id#165, year_total#166] +Input [12]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#141, CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#143, customer_id#167, year_total#168] (110) TakeOrderedAndProject Input [7]: [customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57] @@ -653,7 +653,7 @@ Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_ (114) BroadcastExchange Input [2]: [d_date_sk#17, d_year#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#168] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#170] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#40 IN dynamicpruning#41 BroadcastExchange (118) @@ -678,14 +678,14 @@ Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_ (118) BroadcastExchange Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#169] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#171] Subquery:3 Hosting operator id = 37 Hosting Expression = cs_sold_date_sk#73 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 57 Hosting Expression = cs_sold_date_sk#101 IN dynamicpruning#41 +Subquery:4 Hosting operator id = 57 Hosting Expression = cs_sold_date_sk#102 IN dynamicpruning#41 -Subquery:5 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#127 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#128 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#155 IN dynamicpruning#41 +Subquery:6 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#157 IN dynamicpruning#41 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index 16a3e25049bcc..e267c917700c2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -1,12 +1,12 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] WholeStageCodegen (24) Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id,CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END,year_total,year_total] Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END] + Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] 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 686036a835e25..a612378242f1e 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 @@ -223,61 +223,61 @@ Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_yea Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (rn#39 + 1) AS (v1_lag.rn + 1)#40] Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] (40) Exchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40, 5), ENSURE_REQUIREMENTS, [id=#41] (41) Sort [codegen id : 23] -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] +Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (v1_lag.rn + 1)#40 ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40] Join condition: None (43) Project [codegen id : 24] Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37] -Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] (44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] (45) Sort [codegen id : 33] -Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 (46) Window -Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] -Arguments: [rank(d_year#45, d_moy#46) windowspecdefinition(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#48], [i_category#41, i_brand#42, s_store_name#43, s_company_name#44], [d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST] +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Input [8]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47, rn#48] +Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (rn#49 - 1) AS (v1_lead.rn - 1)#50] +Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] (48) Exchange -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: hashpartitioning(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1), 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] +Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50, 5), ENSURE_REQUIREMENTS, [id=#51] (49) Sort [codegen id : 35] -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, (rn#48 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (v1_lead.rn - 1)#50 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] +Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50] Join condition: None (51) Project [codegen id : 36] -Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] -Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#52, sum_sales#48 AS nsum#53] +Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] (52) TakeOrderedAndProject -Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] +Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt index 07c75d91ca3cf..5c9d3bb5049c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] WholeStageCodegen (36) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,s_store_name,s_company_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,s_store_name,s_company_name,rn] + Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #8 + Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] #8 WholeStageCodegen (22) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,s_store_name,s_company_name,rn] + Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #10 + Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] #10 WholeStageCodegen (34) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter 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 1617f96c6c4d1..6b24837f0cd90 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 @@ -196,53 +196,53 @@ Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_yea Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#26, i_brand#27, s_store_name#28, s_company_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] +Output [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, (rn#37 + 1) AS (v1_lag.rn + 1)#38] Input [8]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] (35) BroadcastExchange -Input [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [id=#38] +Input [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], input[5, int, false]),false), [id=#39] (36) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#24] -Right keys [5]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, (rn#37 + 1)] +Right keys [5]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, (v1_lag.rn + 1)#38] Join condition: None (37) Project [codegen id : 22] Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [15]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] +Input [15]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, (v1_lag.rn + 1)#38] (38) ReusedExchange [Reuses operator id: 31] -Output [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] +Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] (39) Sort [codegen id : 20] -Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, s_store_name#41 ASC NULLS FIRST, s_company_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 (40) Window -Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#39, i_brand#40, s_store_name#41, s_company_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] +Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#40, i_brand#41, s_store_name#42, s_company_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] -Input [8]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] +Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, (rn#47 - 1) AS (v1_lead.rn - 1)#48] +Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] (42) BroadcastExchange -Input [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [id=#47] +Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, (v1_lead.rn - 1)#48] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], input[5, int, false]),false), [id=#49] (43) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#24] -Right keys [5]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, (rn#46 - 1)] +Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (v1_lead.rn - 1)#48] Join condition: None (44) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#48, sum_sales#45 AS nsum#49] -Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] +Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#50, sum_sales#46 AS nsum#51] +Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, (v1_lead.rn - 1)#48] (45) TakeOrderedAndProject -Input [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#48, nsum#49] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#48, nsum#49] +Input [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#50, nsum#51] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#50, nsum#51] ===== Subqueries ===== @@ -269,6 +269,6 @@ Condition : ((((d_year#11 = 1999) OR ((d_year#11 = 1998) AND (d_moy#12 = 12))) O (49) BroadcastExchange Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index 4e91752778013..de328d5c2904d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] WholeStageCodegen (22) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] Filter [avg_monthly_sales,sum_sales] InputAdapter 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 0f22f27fed99b..b7ba9e2c06424 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 @@ -223,61 +223,61 @@ Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sale Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (rn#37 + 1) AS (v1_lag.rn + 1)#38] Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] (40) Exchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38, 5), ENSURE_REQUIREMENTS, [id=#39] (41) Sort [codegen id : 23] -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (v1_lag.rn + 1)#38 ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38] Join condition: None (43) Project [codegen id : 24] Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] (44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] (45) Sort [codegen id : 33] -Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 (46) Window -Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#39, i_brand#40, cc_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#45], [i_category#39, i_brand#40, cc_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] +Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (rn#46 - 1) AS (v1_lead.rn - 1)#47] +Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] (48) Exchange -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: hashpartitioning(i_category#39, i_brand#40, cc_name#41, (rn#45 - 1), 5), ENSURE_REQUIREMENTS, [id=#46] +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] +Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47, 5), ENSURE_REQUIREMENTS, [id=#48] (49) Sort [codegen id : 35] -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, (rn#45 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (v1_lead.rn - 1)#47 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] +Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47] Join condition: None (51) Project [codegen id : 36] -Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] -Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#49, sum_sales#45 AS nsum#50] +Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] (52) TakeOrderedAndProject -Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt index 3bf10f82e6a88..f3ef99b986f71 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] WholeStageCodegen (36) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lead.rn - 1)] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lag.rn + 1)] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,cc_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,cc_name,rn] + Sort [i_category,i_brand,cc_name,(v1_lag.rn + 1)] InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #8 + Exchange [i_category,i_brand,cc_name,(v1_lag.rn + 1)] #8 WholeStageCodegen (22) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,cc_name,rn] + Sort [i_category,i_brand,cc_name,(v1_lead.rn - 1)] InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #10 + Exchange [i_category,i_brand,cc_name,(v1_lead.rn - 1)] #10 WholeStageCodegen (34) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter 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 83ad4872f96c4..e1fd463439072 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 @@ -196,53 +196,53 @@ Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sale Arguments: [rank(d_year#28, d_moy#29) windowspecdefinition(i_category#25, i_brand#26, cc_name#27, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#25, i_brand#26, cc_name#27], [d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] +Output [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, (rn#35 + 1) AS (v1_lag.rn + 1)#36] Input [7]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#33, rn#35] (35) BroadcastExchange -Input [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#36] +Input [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, (v1_lag.rn + 1)#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[4, int, false]),false), [id=#37] (36) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#23] -Right keys [4]: [i_category#25, i_brand#26, cc_name#27, (rn#35 + 1)] +Right keys [4]: [i_category#25, i_brand#26, cc_name#27, (v1_lag.rn + 1)#36] Join condition: None (37) Project [codegen id : 22] Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33] -Input [13]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] +Input [13]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, i_category#25, i_brand#26, cc_name#27, sum_sales#33, (v1_lag.rn + 1)#36] (38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] +Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] (39) Sort [codegen id : 20] -Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] -Arguments: [i_category#37 ASC NULLS FIRST, i_brand#38 ASC NULLS FIRST, cc_name#39 ASC NULLS FIRST, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST], false, 0 (40) Window -Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] -Arguments: [rank(d_year#40, d_moy#41) windowspecdefinition(i_category#37, i_brand#38, cc_name#39, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#43], [i_category#37, i_brand#38, cc_name#39], [d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST] +Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +Arguments: [rank(d_year#41, d_moy#42) windowspecdefinition(i_category#38, i_brand#39, cc_name#40, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] -Input [7]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42, rn#43] +Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, (rn#44 - 1) AS (v1_lead.rn - 1)#45] +Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] (42) BroadcastExchange -Input [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#44] +Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, (v1_lead.rn - 1)#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[4, int, false]),false), [id=#46] (43) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#23] -Right keys [4]: [i_category#37, i_brand#38, cc_name#39, (rn#43 - 1)] +Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (v1_lead.rn - 1)#45] Join condition: None (44) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, sum_sales#33 AS psum#45, sum_sales#42 AS nsum#46] -Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33, i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] +Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, sum_sales#33 AS psum#47, sum_sales#43 AS nsum#48] +Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33, i_category#38, i_brand#39, cc_name#40, sum_sales#43, (v1_lead.rn - 1)#45] (45) TakeOrderedAndProject -Input [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#45, nsum#46] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#45, nsum#46] +Input [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#47, nsum#48] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#47, nsum#48] ===== Subqueries ===== @@ -269,6 +269,6 @@ Condition : ((((d_year#11 = 1999) OR ((d_year#11 = 1998) AND (d_moy#12 = 12))) O (49) BroadcastExchange Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index ad877cc7b61e0..c528a27f75d55 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] WholeStageCodegen (22) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lead.rn - 1)] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lag.rn + 1)] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] Filter [avg_monthly_sales,sum_sales] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index d6b61f63b41a8..05422c6feb00f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -1,63 +1,67 @@ == Physical Plan == -TakeOrderedAndProject (59) -+- * Project (58) - +- * BroadcastHashJoin Inner BuildRight (57) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Filter (26) - : : +- * HashAggregate (25) - : : +- Exchange (24) - : : +- * HashAggregate (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (12) - : : : :- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (11) - : : : +- * Project (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.date_dim (7) - : : +- BroadcastExchange (20) - : : +- * Filter (19) - : : +- * ColumnarToRow (18) - : : +- Scan parquet default.item (17) - : +- BroadcastExchange (40) - : +- * Filter (39) - : +- * HashAggregate (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (32) - : : +- * BroadcastHashJoin Inner BuildRight (31) - : : :- * Filter (29) - : : : +- * ColumnarToRow (28) - : : : +- Scan parquet default.catalog_sales (27) - : : +- ReusedExchange (30) - : +- ReusedExchange (33) - +- BroadcastExchange (56) - +- * Filter (55) - +- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * Project (51) - +- * BroadcastHashJoin Inner BuildRight (50) - :- * Project (48) - : +- * BroadcastHashJoin Inner BuildRight (47) - : :- * Filter (45) - : : +- * ColumnarToRow (44) - : : +- Scan parquet default.web_sales (43) - : +- ReusedExchange (46) - +- ReusedExchange (49) +TakeOrderedAndProject (63) ++- * Project (62) + +- * SortMergeJoin Inner (61) + :- * Project (45) + : +- * SortMergeJoin Inner (44) + : :- * Sort (28) + : : +- * Project (27) + : : +- * Filter (26) + : : +- * HashAggregate (25) + : : +- Exchange (24) + : : +- * HashAggregate (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Project (16) + : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (12) + : : : :- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.date_dim (4) + : : : +- BroadcastExchange (11) + : : : +- * Project (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.date_dim (7) + : : +- BroadcastExchange (20) + : : +- * Filter (19) + : : +- * ColumnarToRow (18) + : : +- Scan parquet default.item (17) + : +- * Sort (43) + : +- * Project (42) + : +- * Filter (41) + : +- * HashAggregate (40) + : +- Exchange (39) + : +- * HashAggregate (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (34) + : : +- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Filter (31) + : : : +- * ColumnarToRow (30) + : : : +- Scan parquet default.catalog_sales (29) + : : +- ReusedExchange (32) + : +- ReusedExchange (35) + +- * Sort (60) + +- * Project (59) + +- * Filter (58) + +- * HashAggregate (57) + +- Exchange (56) + +- * HashAggregate (55) + +- * Project (54) + +- * BroadcastHashJoin Inner BuildRight (53) + :- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Filter (48) + : : +- * ColumnarToRow (47) + : : +- Scan parquet default.web_sales (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet default.store_sales @@ -171,194 +175,210 @@ Results [2]: [i_item_id#13, sum#16] Input [2]: [i_item_id#13, sum#16] Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] -(25) HashAggregate [codegen id : 15] +(25) HashAggregate [codegen id : 5] Input [2]: [i_item_id#13, sum#16] Keys [1]: [i_item_id#13] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#18] Results [2]: [i_item_id#13 AS item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#18,17,2) AS ss_item_rev#20] -(26) Filter [codegen id : 15] +(26) Filter [codegen id : 5] Input [2]: [item_id#19, ss_item_rev#20] Condition : isnotnull(ss_item_rev#20) -(27) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +(27) Project [codegen id : 5] +Output [6]: [item_id#19, ss_item_rev#20, cast(ss_item_rev#20 as decimal(19,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21, cast(ss_item_rev#20 as decimal(20,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22, CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true) AS (0.90BD * ss_items.ss_item_rev)#23, CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true) AS (1.10BD * ss_items.ss_item_rev)#24] +Input [2]: [item_id#19, ss_item_rev#20] + +(28) Sort [codegen id : 5] +Input [6]: [item_id#19, ss_item_rev#20, CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21, CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22, (0.90BD * ss_items.ss_item_rev)#23, (1.10BD * ss_items.ss_item_rev)#24] +Arguments: [item_id#19 ASC NULLS FIRST], false, 0 + +(29) Scan parquet default.catalog_sales +Output [3]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cs_sold_date_sk#27), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +(30) ColumnarToRow [codegen id : 9] +Input [3]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] -(29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#21) +(31) Filter [codegen id : 9] +Input [3]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +Condition : isnotnull(cs_item_sk#25) -(30) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#24] +(32) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#28] -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] +(33) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_sold_date_sk#27] +Right keys [1]: [d_date_sk#28] Join condition: None -(32) Project [codegen id : 8] -Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] +(34) Project [codegen id : 9] +Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] -(33) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#25, i_item_id#26] +(35) ReusedExchange [Reuses operator id: 20] +Output [2]: [i_item_sk#29, i_item_id#30] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#25] +(36) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_item_sk#25] +Right keys [1]: [i_item_sk#29] Join condition: None -(35) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#26] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#25, i_item_id#26] - -(36) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#26, sum#28] - -(37) Exchange -Input [2]: [i_item_id#26, sum#28] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [id=#29] +(37) Project [codegen id : 9] +Output [2]: [cs_ext_sales_price#26, i_item_id#30] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#29, i_item_id#30] (38) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#26, sum#28] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#26 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] - -(39) Filter [codegen id : 9] -Input [2]: [item_id#31, cs_item_rev#32] -Condition : isnotnull(cs_item_rev#32) - -(40) BroadcastExchange -Input [2]: [item_id#31, cs_item_rev#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] - -(41) BroadcastHashJoin [codegen id : 15] +Input [2]: [cs_ext_sales_price#26, i_item_id#30] +Keys [1]: [i_item_id#30] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] +Aggregate Attributes [1]: [sum#31] +Results [2]: [i_item_id#30, sum#32] + +(39) Exchange +Input [2]: [i_item_id#30, sum#32] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] + +(40) HashAggregate [codegen id : 10] +Input [2]: [i_item_id#30, sum#32] +Keys [1]: [i_item_id#30] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#34] +Results [2]: [i_item_id#30 AS item_id#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#34,17,2) AS cs_item_rev#36] + +(41) Filter [codegen id : 10] +Input [2]: [item_id#35, cs_item_rev#36] +Condition : isnotnull(cs_item_rev#36) + +(42) Project [codegen id : 10] +Output [6]: [item_id#35, cs_item_rev#36, CheckOverflow((0.90 * promote_precision(cs_item_rev#36)), DecimalType(19,3), true) AS (0.90BD * cs_items.cs_item_rev)#37, CheckOverflow((1.10 * promote_precision(cs_item_rev#36)), DecimalType(20,3), true) AS (1.10BD * cs_items.cs_item_rev)#38, cast(cs_item_rev#36 as decimal(19,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39, cast(cs_item_rev#36 as decimal(20,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40] +Input [2]: [item_id#35, cs_item_rev#36] + +(43) Sort [codegen id : 10] +Input [6]: [item_id#35, cs_item_rev#36, (0.90BD * cs_items.cs_item_rev)#37, (1.10BD * cs_items.cs_item_rev)#38, CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39, CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40] +Arguments: [item_id#35 ASC NULLS FIRST], false, 0 + +(44) SortMergeJoin [codegen id : 11] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Right keys [1]: [item_id#35] +Join condition: ((((CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21 >= (0.90BD * cs_items.cs_item_rev)#37) AND (CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22 <= (1.10BD * cs_items.cs_item_rev)#38)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39 >= (0.90BD * ss_items.ss_item_rev)#23)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40 <= (1.10BD * ss_items.ss_item_rev)#24)) -(42) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] -Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] +(45) Project [codegen id : 11] +Output [11]: [item_id#19, ss_item_rev#20, cs_item_rev#36, cast(ss_item_rev#20 as decimal(19,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#41, cast(ss_item_rev#20 as decimal(20,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#42, cast(cs_item_rev#36 as decimal(19,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#43, cast(cs_item_rev#36 as decimal(20,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#44, CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true) AS (0.90BD * ss_items.ss_item_rev)#45, CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true) AS (1.10BD * ss_items.ss_item_rev)#46, CheckOverflow((0.90 * promote_precision(cs_item_rev#36)), DecimalType(19,3), true) AS (0.90BD * cs_items.cs_item_rev)#47, CheckOverflow((1.10 * promote_precision(cs_item_rev#36)), DecimalType(20,3), true) AS (1.10BD * cs_items.cs_item_rev)#48] +Input [12]: [item_id#19, ss_item_rev#20, CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21, CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22, (0.90BD * ss_items.ss_item_rev)#23, (1.10BD * ss_items.ss_item_rev)#24, item_id#35, cs_item_rev#36, (0.90BD * cs_items.cs_item_rev)#37, (1.10BD * cs_items.cs_item_rev)#38, CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39, CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40] -(43) Scan parquet default.web_sales -Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(46) Scan parquet default.web_sales +Output [3]: [ws_item_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(47) ColumnarToRow [codegen id : 15] +Input [3]: [ws_item_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -(45) Filter [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#34) +(48) Filter [codegen id : 15] +Input [3]: [ws_item_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_item_sk#49) -(46) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#37] +(49) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#52] -(47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#37] +(50) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#52] Join condition: None -(48) Project [codegen id : 13] -Output [2]: [ws_item_sk#34, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#37] +(51) Project [codegen id : 15] +Output [2]: [ws_item_sk#49, ws_ext_sales_price#50] +Input [4]: [ws_item_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51, d_date_sk#52] -(49) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#38, i_item_id#39] +(52) ReusedExchange [Reuses operator id: 20] +Output [2]: [i_item_sk#53, i_item_id#54] -(50) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#38] +(53) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#49] +Right keys [1]: [i_item_sk#53] Join condition: None -(51) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#35, i_item_id#39] -Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, i_item_sk#38, i_item_id#39] - -(52) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#35, i_item_id#39] -Keys [1]: [i_item_id#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#39, sum#41] - -(53) Exchange -Input [2]: [i_item_id#39, sum#41] -Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, [id=#42] - -(54) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#39, sum#41] -Keys [1]: [i_item_id#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#39 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] - -(55) Filter [codegen id : 14] -Input [2]: [item_id#44, ws_item_rev#45] -Condition : isnotnull(ws_item_rev#45) - -(56) BroadcastExchange -Input [2]: [item_id#44, ws_item_rev#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] - -(57) BroadcastHashJoin [codegen id : 15] +(54) Project [codegen id : 15] +Output [2]: [ws_ext_sales_price#50, i_item_id#54] +Input [4]: [ws_item_sk#49, ws_ext_sales_price#50, i_item_sk#53, i_item_id#54] + +(55) HashAggregate [codegen id : 15] +Input [2]: [ws_ext_sales_price#50, i_item_id#54] +Keys [1]: [i_item_id#54] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#50))] +Aggregate Attributes [1]: [sum#55] +Results [2]: [i_item_id#54, sum#56] + +(56) Exchange +Input [2]: [i_item_id#54, sum#56] +Arguments: hashpartitioning(i_item_id#54, 5), ENSURE_REQUIREMENTS, [id=#57] + +(57) HashAggregate [codegen id : 16] +Input [2]: [i_item_id#54, sum#56] +Keys [1]: [i_item_id#54] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#50))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#50))#58] +Results [2]: [i_item_id#54 AS item_id#59, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#50))#58,17,2) AS ws_item_rev#60] + +(58) Filter [codegen id : 16] +Input [2]: [item_id#59, ws_item_rev#60] +Condition : isnotnull(ws_item_rev#60) + +(59) Project [codegen id : 16] +Output [6]: [item_id#59, ws_item_rev#60, CheckOverflow((0.90 * promote_precision(ws_item_rev#60)), DecimalType(19,3), true) AS (0.90BD * ws_items.ws_item_rev)#61, CheckOverflow((1.10 * promote_precision(ws_item_rev#60)), DecimalType(20,3), true) AS (1.10BD * ws_items.ws_item_rev)#62, cast(ws_item_rev#60 as decimal(19,3)) AS CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#63, cast(ws_item_rev#60 as decimal(20,3)) AS CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#64] +Input [2]: [item_id#59, ws_item_rev#60] + +(60) Sort [codegen id : 16] +Input [6]: [item_id#59, ws_item_rev#60, (0.90BD * ws_items.ws_item_rev)#61, (1.10BD * ws_items.ws_item_rev)#62, CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#63, CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#64] +Arguments: [item_id#59 ASC NULLS FIRST], false, 0 + +(61) SortMergeJoin [codegen id : 17] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) +Right keys [1]: [item_id#59] +Join condition: ((((((((CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#41 >= (0.90BD * ws_items.ws_item_rev)#61) AND (CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#42 <= (1.10BD * ws_items.ws_item_rev)#62)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#43 >= (0.90BD * ws_items.ws_item_rev)#61)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#44 <= (1.10BD * ws_items.ws_item_rev)#62)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#63 >= (0.90BD * ss_items.ss_item_rev)#45)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#64 <= (1.10BD * ss_items.ss_item_rev)#46)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#63 >= (0.90BD * cs_items.cs_item_rev)#47)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#64 <= (1.10BD * cs_items.cs_item_rev)#48)) -(58) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] +(62) Project [codegen id : 17] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#60 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#65, cs_item_rev#36, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#36 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#60 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#66, ws_item_rev#60, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#60 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#60 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#67, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#60 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#68] +Input [17]: [item_id#19, ss_item_rev#20, cs_item_rev#36, CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#41, CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#42, CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#43, CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#44, (0.90BD * ss_items.ss_item_rev)#45, (1.10BD * ss_items.ss_item_rev)#46, (0.90BD * cs_items.cs_item_rev)#47, (1.10BD * cs_items.cs_item_rev)#48, item_id#59, ws_item_rev#60, (0.90BD * ws_items.ws_item_rev)#61, (1.10BD * ws_items.ws_item_rev)#62, CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#63, CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#64] -(59) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +(63) TakeOrderedAndProject +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#65, cs_item_rev#36, cs_dev#66, ws_item_rev#60, ws_dev#67, average#68] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#65, cs_item_rev#36, cs_dev#66, ws_item_rev#60, ws_dev#67, average#68] ===== Subqueries ===== Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* Project (63) -+- * Filter (62) - +- * ColumnarToRow (61) - +- Scan parquet default.date_dim (60) +* Project (67) ++- * Filter (66) + +- * ColumnarToRow (65) + +- Scan parquet default.date_dim (64) -(60) Scan parquet default.date_dim -Output [2]: [d_date#51, d_week_seq#52] +(64) Scan parquet default.date_dim +Output [2]: [d_date#69, d_week_seq#70] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#51, d_week_seq#52] +(65) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#69, d_week_seq#70] -(62) Filter [codegen id : 1] -Input [2]: [d_date#51, d_week_seq#52] -Condition : (isnotnull(d_date#51) AND (d_date#51 = 2000-01-03)) +(66) Filter [codegen id : 1] +Input [2]: [d_date#69, d_week_seq#70] +Condition : (isnotnull(d_date#69) AND (d_date#69 = 2000-01-03)) -(63) Project [codegen id : 1] -Output [1]: [d_week_seq#52] -Input [2]: [d_date#51, d_week_seq#52] +(67) Project [codegen id : 1] +Output [1]: [d_week_seq#70] +Input [2]: [d_date#69, d_week_seq#70] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt index 57c79b9aee840..52eafb3dfdfa1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt @@ -1,93 +1,101 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - WholeStageCodegen (15) + WholeStageCodegen (17) Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] - Project [item_id,ss_item_rev,cs_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] - Filter [ss_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] + SortMergeJoin [item_id,item_id,CAST(ss_items.ss_item_rev AS DECIMAL(19,3)),(0.90BD * ws_items.ws_item_rev),CAST(ss_items.ss_item_rev AS DECIMAL(20,3)),(1.10BD * ws_items.ws_item_rev),CAST(cs_items.cs_item_rev AS DECIMAL(19,3)),CAST(cs_items.cs_item_rev AS DECIMAL(20,3)),CAST(ws_items.ws_item_rev AS DECIMAL(19,3)),(0.90BD * ss_items.ss_item_rev),CAST(ws_items.ws_item_rev AS DECIMAL(20,3)),(1.10BD * ss_items.ss_item_rev),(0.90BD * cs_items.cs_item_rev),(1.10BD * cs_items.cs_item_rev)] + InputAdapter + WholeStageCodegen (11) + Project [item_id,ss_item_rev,cs_item_rev] + SortMergeJoin [item_id,item_id,CAST(ss_items.ss_item_rev AS DECIMAL(19,3)),(0.90BD * cs_items.cs_item_rev),CAST(ss_items.ss_item_rev AS DECIMAL(20,3)),(1.10BD * cs_items.cs_item_rev),CAST(cs_items.cs_item_rev AS DECIMAL(19,3)),(0.90BD * ss_items.ss_item_rev),CAST(cs_items.cs_item_rev AS DECIMAL(20,3)),(1.10BD * ss_items.ss_item_rev)] InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date] - Filter [d_week_seq] - Subquery #1 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - ColumnarToRow + WholeStageCodegen (5) + Sort [item_id] + Project [item_id,ss_item_rev] + Filter [ss_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + Filter [d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date] + Filter [d_week_seq] + Subquery #1 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + Filter [i_item_sk,i_item_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id] + InputAdapter + WholeStageCodegen (10) + Sort [item_id] + Project [item_id,cs_item_rev] + Filter [cs_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [i_item_sk,i_item_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (9) - Filter [cs_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - InputAdapter - Exchange [i_item_id] #6 - WholeStageCodegen (8) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 + Exchange [i_item_id] #5 + WholeStageCodegen (9) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (14) - Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (13) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 + WholeStageCodegen (16) + Sort [item_id] + Project [item_id,ws_item_rev] + Filter [ws_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] + InputAdapter + Exchange [i_item_id] #6 + WholeStageCodegen (15) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index a8ce33ee99d7b..47a8eab8bd685 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -1,63 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (59) -+- * Project (58) - +- * BroadcastHashJoin Inner BuildRight (57) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Filter (26) - : : +- * HashAggregate (25) - : : +- Exchange (24) - : : +- * HashAggregate (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin LeftSemi BuildRight (18) - : : :- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (17) - : : +- * Project (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.date_dim (13) - : +- BroadcastExchange (40) - : +- * Filter (39) - : +- * HashAggregate (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (32) - : : +- * BroadcastHashJoin Inner BuildRight (31) - : : :- * Filter (29) - : : : +- * ColumnarToRow (28) - : : : +- Scan parquet default.catalog_sales (27) - : : +- ReusedExchange (30) - : +- ReusedExchange (33) - +- BroadcastExchange (56) - +- * Filter (55) - +- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * Project (51) - +- * BroadcastHashJoin Inner BuildRight (50) - :- * Project (48) - : +- * BroadcastHashJoin Inner BuildRight (47) - : :- * Filter (45) - : : +- * ColumnarToRow (44) - : : +- Scan parquet default.web_sales (43) - : +- ReusedExchange (46) - +- ReusedExchange (49) +TakeOrderedAndProject (62) ++- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (27) + : : +- * Filter (26) + : : +- * HashAggregate (25) + : : +- Exchange (24) + : : +- * HashAggregate (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.item (4) + : : +- BroadcastExchange (20) + : : +- * Project (19) + : : +- * BroadcastHashJoin LeftSemi BuildRight (18) + : : :- * Filter (12) + : : : +- * ColumnarToRow (11) + : : : +- Scan parquet default.date_dim (10) + : : +- BroadcastExchange (17) + : : +- * Project (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.date_dim (13) + : +- BroadcastExchange (42) + : +- * Project (41) + : +- * Filter (40) + : +- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Project (33) + : : +- * BroadcastHashJoin Inner BuildRight (32) + : : :- * Filter (30) + : : : +- * ColumnarToRow (29) + : : : +- Scan parquet default.catalog_sales (28) + : : +- ReusedExchange (31) + : +- ReusedExchange (34) + +- BroadcastExchange (59) + +- * Project (58) + +- * Filter (57) + +- * HashAggregate (56) + +- Exchange (55) + +- * HashAggregate (54) + +- * Project (53) + +- * BroadcastHashJoin Inner BuildRight (52) + :- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) + : :- * Filter (47) + : : +- * ColumnarToRow (46) + : : +- Scan parquet default.web_sales (45) + : +- ReusedExchange (48) + +- ReusedExchange (51) (1) Scan parquet default.store_sales @@ -182,183 +185,195 @@ Results [2]: [i_item_id#5 AS item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sa Input [2]: [item_id#19, ss_item_rev#20] Condition : isnotnull(ss_item_rev#20) -(27) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +(27) Project [codegen id : 15] +Output [6]: [item_id#19, ss_item_rev#20, cast(ss_item_rev#20 as decimal(19,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21, cast(ss_item_rev#20 as decimal(20,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22, CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true) AS (0.90BD * ss_items.ss_item_rev)#23, CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true) AS (1.10BD * ss_items.ss_item_rev)#24] +Input [2]: [item_id#19, ss_item_rev#20] + +(28) Scan parquet default.catalog_sales +Output [3]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cs_sold_date_sk#27), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +(29) ColumnarToRow [codegen id : 8] +Input [3]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] -(29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#21) +(30) Filter [codegen id : 8] +Input [3]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +Condition : isnotnull(cs_item_sk#25) -(30) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#24, i_item_id#25] +(31) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#28, i_item_id#29] -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#24] +(32) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#25] +Right keys [1]: [i_item_sk#28] Join condition: None -(32) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25] -Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#24, i_item_id#25] +(33) Project [codegen id : 8] +Output [3]: [cs_ext_sales_price#26, cs_sold_date_sk#27, i_item_id#29] +Input [5]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, i_item_sk#28, i_item_id#29] -(33) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#26] +(34) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#30] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#26] +(35) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#27] +Right keys [1]: [d_date_sk#30] Join condition: None -(35) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#25] -Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25, d_date_sk#26] - -(36) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#25] -Keys [1]: [i_item_id#25] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#25, sum#28] - -(37) Exchange -Input [2]: [i_item_id#25, sum#28] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#29] - -(38) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#25, sum#28] -Keys [1]: [i_item_id#25] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#25 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] - -(39) Filter [codegen id : 9] -Input [2]: [item_id#31, cs_item_rev#32] -Condition : isnotnull(cs_item_rev#32) - -(40) BroadcastExchange -Input [2]: [item_id#31, cs_item_rev#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] - -(41) BroadcastHashJoin [codegen id : 15] +(36) Project [codegen id : 8] +Output [2]: [cs_ext_sales_price#26, i_item_id#29] +Input [4]: [cs_ext_sales_price#26, cs_sold_date_sk#27, i_item_id#29, d_date_sk#30] + +(37) HashAggregate [codegen id : 8] +Input [2]: [cs_ext_sales_price#26, i_item_id#29] +Keys [1]: [i_item_id#29] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] +Aggregate Attributes [1]: [sum#31] +Results [2]: [i_item_id#29, sum#32] + +(38) Exchange +Input [2]: [i_item_id#29, sum#32] +Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, [id=#33] + +(39) HashAggregate [codegen id : 9] +Input [2]: [i_item_id#29, sum#32] +Keys [1]: [i_item_id#29] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#34] +Results [2]: [i_item_id#29 AS item_id#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#34,17,2) AS cs_item_rev#36] + +(40) Filter [codegen id : 9] +Input [2]: [item_id#35, cs_item_rev#36] +Condition : isnotnull(cs_item_rev#36) + +(41) Project [codegen id : 9] +Output [6]: [item_id#35, cs_item_rev#36, CheckOverflow((0.90 * promote_precision(cs_item_rev#36)), DecimalType(19,3), true) AS (0.90BD * cs_items.cs_item_rev)#37, CheckOverflow((1.10 * promote_precision(cs_item_rev#36)), DecimalType(20,3), true) AS (1.10BD * cs_items.cs_item_rev)#38, cast(cs_item_rev#36 as decimal(19,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39, cast(cs_item_rev#36 as decimal(20,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40] +Input [2]: [item_id#35, cs_item_rev#36] + +(42) BroadcastExchange +Input [6]: [item_id#35, cs_item_rev#36, (0.90BD * cs_items.cs_item_rev)#37, (1.10BD * cs_items.cs_item_rev)#38, CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39, CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#41] + +(43) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Right keys [1]: [item_id#35] +Join condition: ((((CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21 >= (0.90BD * cs_items.cs_item_rev)#37) AND (CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22 <= (1.10BD * cs_items.cs_item_rev)#38)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39 >= (0.90BD * ss_items.ss_item_rev)#23)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40 <= (1.10BD * ss_items.ss_item_rev)#24)) -(42) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] -Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] +(44) Project [codegen id : 15] +Output [11]: [item_id#19, ss_item_rev#20, cs_item_rev#36, cast(ss_item_rev#20 as decimal(19,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#42, cast(ss_item_rev#20 as decimal(20,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#43, cast(cs_item_rev#36 as decimal(19,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#44, cast(cs_item_rev#36 as decimal(20,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#45, CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true) AS (0.90BD * ss_items.ss_item_rev)#46, CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true) AS (1.10BD * ss_items.ss_item_rev)#47, CheckOverflow((0.90 * promote_precision(cs_item_rev#36)), DecimalType(19,3), true) AS (0.90BD * cs_items.cs_item_rev)#48, CheckOverflow((1.10 * promote_precision(cs_item_rev#36)), DecimalType(20,3), true) AS (1.10BD * cs_items.cs_item_rev)#49] +Input [12]: [item_id#19, ss_item_rev#20, CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21, CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22, (0.90BD * ss_items.ss_item_rev)#23, (1.10BD * ss_items.ss_item_rev)#24, item_id#35, cs_item_rev#36, (0.90BD * cs_items.cs_item_rev)#37, (1.10BD * cs_items.cs_item_rev)#38, CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39, CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40] -(43) Scan parquet default.web_sales -Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(45) Scan parquet default.web_sales +Output [3]: [ws_item_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(46) ColumnarToRow [codegen id : 13] +Input [3]: [ws_item_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] -(45) Filter [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#34) +(47) Filter [codegen id : 13] +Input [3]: [ws_item_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] +Condition : isnotnull(ws_item_sk#50) -(46) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#37, i_item_id#38] +(48) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#53, i_item_id#54] -(47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#37] +(49) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_item_sk#50] +Right keys [1]: [i_item_sk#53] Join condition: None -(48) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#38] -Input [5]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_sk#37, i_item_id#38] +(50) Project [codegen id : 13] +Output [3]: [ws_ext_sales_price#51, ws_sold_date_sk#52, i_item_id#54] +Input [5]: [ws_item_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52, i_item_sk#53, i_item_id#54] -(49) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#39] +(51) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#55] -(50) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#39] +(52) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_sold_date_sk#52] +Right keys [1]: [d_date_sk#55] Join condition: None -(51) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#35, i_item_id#38] -Input [4]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#38, d_date_sk#39] - -(52) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#35, i_item_id#38] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#38, sum#41] - -(53) Exchange -Input [2]: [i_item_id#38, sum#41] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [id=#42] - -(54) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#38, sum#41] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#38 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] - -(55) Filter [codegen id : 14] -Input [2]: [item_id#44, ws_item_rev#45] -Condition : isnotnull(ws_item_rev#45) - -(56) BroadcastExchange -Input [2]: [item_id#44, ws_item_rev#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] - -(57) BroadcastHashJoin [codegen id : 15] +(53) Project [codegen id : 13] +Output [2]: [ws_ext_sales_price#51, i_item_id#54] +Input [4]: [ws_ext_sales_price#51, ws_sold_date_sk#52, i_item_id#54, d_date_sk#55] + +(54) HashAggregate [codegen id : 13] +Input [2]: [ws_ext_sales_price#51, i_item_id#54] +Keys [1]: [i_item_id#54] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#51))] +Aggregate Attributes [1]: [sum#56] +Results [2]: [i_item_id#54, sum#57] + +(55) Exchange +Input [2]: [i_item_id#54, sum#57] +Arguments: hashpartitioning(i_item_id#54, 5), ENSURE_REQUIREMENTS, [id=#58] + +(56) HashAggregate [codegen id : 14] +Input [2]: [i_item_id#54, sum#57] +Keys [1]: [i_item_id#54] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#51))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#51))#59] +Results [2]: [i_item_id#54 AS item_id#60, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#51))#59,17,2) AS ws_item_rev#61] + +(57) Filter [codegen id : 14] +Input [2]: [item_id#60, ws_item_rev#61] +Condition : isnotnull(ws_item_rev#61) + +(58) Project [codegen id : 14] +Output [6]: [item_id#60, ws_item_rev#61, CheckOverflow((0.90 * promote_precision(ws_item_rev#61)), DecimalType(19,3), true) AS (0.90BD * ws_items.ws_item_rev)#62, CheckOverflow((1.10 * promote_precision(ws_item_rev#61)), DecimalType(20,3), true) AS (1.10BD * ws_items.ws_item_rev)#63, cast(ws_item_rev#61 as decimal(19,3)) AS CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#64, cast(ws_item_rev#61 as decimal(20,3)) AS CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#65] +Input [2]: [item_id#60, ws_item_rev#61] + +(59) BroadcastExchange +Input [6]: [item_id#60, ws_item_rev#61, (0.90BD * ws_items.ws_item_rev)#62, (1.10BD * ws_items.ws_item_rev)#63, CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#64, CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#65] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#66] + +(60) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) +Right keys [1]: [item_id#60] +Join condition: ((((((((CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#42 >= (0.90BD * ws_items.ws_item_rev)#62) AND (CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#43 <= (1.10BD * ws_items.ws_item_rev)#63)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#44 >= (0.90BD * ws_items.ws_item_rev)#62)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#45 <= (1.10BD * ws_items.ws_item_rev)#63)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#64 >= (0.90BD * ss_items.ss_item_rev)#46)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#65 <= (1.10BD * ss_items.ss_item_rev)#47)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#64 >= (0.90BD * cs_items.cs_item_rev)#48)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#65 <= (1.10BD * cs_items.cs_item_rev)#49)) -(58) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] +(61) Project [codegen id : 15] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#61 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#67, cs_item_rev#36, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#36 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#61 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#68, ws_item_rev#61, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#61 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#61 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#69, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#61 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#70] +Input [17]: [item_id#19, ss_item_rev#20, cs_item_rev#36, CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#42, CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#43, CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#44, CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#45, (0.90BD * ss_items.ss_item_rev)#46, (1.10BD * ss_items.ss_item_rev)#47, (0.90BD * cs_items.cs_item_rev)#48, (1.10BD * cs_items.cs_item_rev)#49, item_id#60, ws_item_rev#61, (0.90BD * ws_items.ws_item_rev)#62, (1.10BD * ws_items.ws_item_rev)#63, CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#64, CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#65] -(59) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +(62) TakeOrderedAndProject +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#67, cs_item_rev#36, cs_dev#68, ws_item_rev#61, ws_dev#69, average#70] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#67, cs_item_rev#36, cs_dev#68, ws_item_rev#61, ws_dev#69, average#70] ===== Subqueries ===== Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (63) -+- * Filter (62) - +- * ColumnarToRow (61) - +- Scan parquet default.date_dim (60) +* Project (66) ++- * Filter (65) + +- * ColumnarToRow (64) + +- Scan parquet default.date_dim (63) -(60) Scan parquet default.date_dim -Output [2]: [d_date#51, d_week_seq#52] +(63) Scan parquet default.date_dim +Output [2]: [d_date#71, d_week_seq#72] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#51, d_week_seq#52] +(64) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#71, d_week_seq#72] -(62) Filter [codegen id : 1] -Input [2]: [d_date#51, d_week_seq#52] -Condition : (isnotnull(d_date#51) AND (d_date#51 = 2000-01-03)) +(65) Filter [codegen id : 1] +Input [2]: [d_date#71, d_week_seq#72] +Condition : (isnotnull(d_date#71) AND (d_date#71 = 2000-01-03)) -(63) Project [codegen id : 1] -Output [1]: [d_week_seq#52] -Input [2]: [d_date#51, d_week_seq#52] +(66) Project [codegen id : 1] +Output [1]: [d_week_seq#72] +Input [2]: [d_date#71, d_week_seq#72] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 1fb6832ea9b4a..7e934ad382a09 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -1,93 +1,96 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] WholeStageCodegen (15) Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] + BroadcastHashJoin [item_id,item_id,CAST(ss_items.ss_item_rev AS DECIMAL(19,3)),(0.90BD * ws_items.ws_item_rev),CAST(ss_items.ss_item_rev AS DECIMAL(20,3)),(1.10BD * ws_items.ws_item_rev),CAST(cs_items.cs_item_rev AS DECIMAL(19,3)),CAST(cs_items.cs_item_rev AS DECIMAL(20,3)),CAST(ws_items.ws_item_rev AS DECIMAL(19,3)),(0.90BD * ss_items.ss_item_rev),CAST(ws_items.ws_item_rev AS DECIMAL(20,3)),(1.10BD * ss_items.ss_item_rev),(0.90BD * cs_items.cs_item_rev),(1.10BD * cs_items.cs_item_rev)] Project [item_id,ss_item_rev,cs_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] - Filter [ss_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [i_item_sk,i_item_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + BroadcastHashJoin [item_id,item_id,CAST(ss_items.ss_item_rev AS DECIMAL(19,3)),(0.90BD * cs_items.cs_item_rev),CAST(ss_items.ss_item_rev AS DECIMAL(20,3)),(1.10BD * cs_items.cs_item_rev),CAST(cs_items.cs_item_rev AS DECIMAL(19,3)),(0.90BD * ss_items.ss_item_rev),CAST(cs_items.cs_item_rev AS DECIMAL(20,3)),(1.10BD * ss_items.ss_item_rev)] + Project [item_id,ss_item_rev] + Filter [ss_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date] - Filter [d_week_seq] - Subquery #1 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + Filter [i_item_sk,i_item_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (3) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + Filter [d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Project [d_date] + Filter [d_week_seq] + Subquery #1 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Filter [cs_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - InputAdapter - Exchange [i_item_id] #6 - WholeStageCodegen (8) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 - InputAdapter - ReusedExchange [d_date_sk] #3 + Project [item_id,cs_item_rev] + Filter [cs_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] + InputAdapter + Exchange [i_item_id] #6 + WholeStageCodegen (8) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #2 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #7 WholeStageCodegen (14) - Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (13) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 - InputAdapter - ReusedExchange [d_date_sk] #3 + Project [item_id,ws_item_rev] + Filter [ws_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] + InputAdapter + Exchange [i_item_id] #8 + WholeStageCodegen (13) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #2 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt index cf22f178dd33f..c07e395777b7d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt @@ -228,21 +228,21 @@ Right keys [1]: [d_week_seq#76] Join condition: None (40) Project [codegen id : 9] -Output [9]: [d_week_seq#54 AS d_week_seq2#78, s_store_id#73 AS s_store_id2#79, sun_sales#30 AS sun_sales2#80, mon_sales#31 AS mon_sales2#81, tue_sales#32 AS tue_sales2#82, wed_sales#33 AS wed_sales2#83, thu_sales#34 AS thu_sales2#84, fri_sales#35 AS fri_sales2#85, sat_sales#36 AS sat_sales2#86] +Output [9]: [s_store_id#73 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, tue_sales#32 AS tue_sales2#81, wed_sales#33 AS wed_sales2#82, thu_sales#34 AS thu_sales2#83, fri_sales#35 AS fri_sales2#84, sat_sales#36 AS sat_sales2#85, (d_week_seq#54 - 52) AS (x.d_week_seq2 - 52)#86] Input [10]: [d_week_seq#54, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#73, d_week_seq#76] (41) BroadcastExchange -Input [9]: [d_week_seq2#78, s_store_id2#79, sun_sales2#80, mon_sales2#81, tue_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#87] +Input [9]: [s_store_id2#78, sun_sales2#79, mon_sales2#80, tue_sales2#81, wed_sales2#82, thu_sales2#83, fri_sales2#84, sat_sales2#85, (x.d_week_seq2 - 52)#86] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[8, int, true]),false), [id=#87] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#79, (d_week_seq2#78 - 52)] +Right keys [2]: [s_store_id2#78, (x.d_week_seq2 - 52)#86] Join condition: None (43) Project [codegen id : 10] -Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#80)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#88, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#81)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#89, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#82)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#90, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#83)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#91, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#84)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#92, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#85)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#93, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#86)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#94] -Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#78, s_store_id2#79, sun_sales2#80, mon_sales2#81, tue_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] +Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#88, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#89, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#81)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#90, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#82)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#91, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#83)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#92, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#84)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#93, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#85)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#94] +Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, s_store_id2#78, sun_sales2#79, mon_sales2#80, tue_sales2#81, wed_sales2#82, thu_sales2#83, fri_sales2#84, sat_sales2#85, (x.d_week_seq2 - 52)#86] (44) TakeOrderedAndProject Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#88, (mon_sales1 / mon_sales2)#89, (tue_sales1 / tue_sales2)#90, (wed_sales1 / wed_sales2)#91, (thu_sales1 / thu_sales2)#92, (fri_sales1 / fri_sales2)#93, (sat_sales1 / sat_sales2)#94] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt index 541817641b129..99748e7fd3233 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] WholeStageCodegen (10) Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] + BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,(x.d_week_seq2 - 52)] Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] @@ -42,7 +42,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Project [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + Project [s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt index cf22f178dd33f..c07e395777b7d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt @@ -228,21 +228,21 @@ Right keys [1]: [d_week_seq#76] Join condition: None (40) Project [codegen id : 9] -Output [9]: [d_week_seq#54 AS d_week_seq2#78, s_store_id#73 AS s_store_id2#79, sun_sales#30 AS sun_sales2#80, mon_sales#31 AS mon_sales2#81, tue_sales#32 AS tue_sales2#82, wed_sales#33 AS wed_sales2#83, thu_sales#34 AS thu_sales2#84, fri_sales#35 AS fri_sales2#85, sat_sales#36 AS sat_sales2#86] +Output [9]: [s_store_id#73 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, tue_sales#32 AS tue_sales2#81, wed_sales#33 AS wed_sales2#82, thu_sales#34 AS thu_sales2#83, fri_sales#35 AS fri_sales2#84, sat_sales#36 AS sat_sales2#85, (d_week_seq#54 - 52) AS (x.d_week_seq2 - 52)#86] Input [10]: [d_week_seq#54, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#73, d_week_seq#76] (41) BroadcastExchange -Input [9]: [d_week_seq2#78, s_store_id2#79, sun_sales2#80, mon_sales2#81, tue_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#87] +Input [9]: [s_store_id2#78, sun_sales2#79, mon_sales2#80, tue_sales2#81, wed_sales2#82, thu_sales2#83, fri_sales2#84, sat_sales2#85, (x.d_week_seq2 - 52)#86] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[8, int, true]),false), [id=#87] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#79, (d_week_seq2#78 - 52)] +Right keys [2]: [s_store_id2#78, (x.d_week_seq2 - 52)#86] Join condition: None (43) Project [codegen id : 10] -Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#80)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#88, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#81)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#89, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#82)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#90, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#83)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#91, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#84)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#92, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#85)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#93, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#86)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#94] -Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#78, s_store_id2#79, sun_sales2#80, mon_sales2#81, tue_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] +Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#88, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#89, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#81)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#90, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#82)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#91, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#83)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#92, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#84)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#93, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#85)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#94] +Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, s_store_id2#78, sun_sales2#79, mon_sales2#80, tue_sales2#81, wed_sales2#82, thu_sales2#83, fri_sales2#84, sat_sales2#85, (x.d_week_seq2 - 52)#86] (44) TakeOrderedAndProject Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#88, (mon_sales1 / mon_sales2)#89, (tue_sales1 / tue_sales2)#90, (wed_sales1 / wed_sales2)#91, (thu_sales1 / thu_sales2)#92, (fri_sales1 / fri_sales2)#93, (sat_sales1 / sat_sales2)#94] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt index 541817641b129..99748e7fd3233 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] WholeStageCodegen (10) Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] + BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,(x.d_week_seq2 - 52)] Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] @@ -42,7 +42,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Project [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + Project [s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt index e4baf3b296016..16850fd1593b6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt @@ -1,45 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- * SortMergeJoin Inner (39) - :- * Sort (33) - : +- Exchange (32) - : +- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Filter (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (23) - : : +- * HashAggregate (22) - : : +- Exchange (21) - : : +- * HashAggregate (20) - : : +- * HashAggregate (19) - : : +- Exchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.store_sales (11) - : : +- ReusedExchange (14) - : +- BroadcastExchange (29) - : +- * Filter (28) - : +- * ColumnarToRow (27) - : +- Scan parquet default.store (26) - +- * Sort (38) - +- Exchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.item (34) +TakeOrderedAndProject (42) ++- * Project (41) + +- * SortMergeJoin Inner (40) + :- * Sort (34) + : +- Exchange (33) + : +- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (11) + : : : +- * Filter (10) + : : : +- * HashAggregate (9) + : : : +- Exchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (24) + : : +- * HashAggregate (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- * HashAggregate (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Filter (14) + : : : +- * ColumnarToRow (13) + : : : +- Scan parquet default.store_sales (12) + : : +- ReusedExchange (15) + : +- BroadcastExchange (30) + : +- * Filter (29) + : +- * ColumnarToRow (28) + : +- Scan parquet default.store (27) + +- * Sort (39) + +- Exchange (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.item (35) (1) Scan parquet default.store_sales @@ -57,7 +58,7 @@ Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4] Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) -(4) ReusedExchange [Reuses operator id: 46] +(4) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 2] @@ -91,184 +92,188 @@ Results [3]: [ss_store_sk#2, ss_item_sk#1, MakeDecimal(sum(UnscaledValue(ss_sale Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] Condition : isnotnull(revenue#11) -(11) Scan parquet default.store_sales -Output [4]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +(11) Project [codegen id : 8] +Output [4]: [ss_store_sk#2, ss_item_sk#1, revenue#11, cast(revenue#11 as decimal(23,7)) AS CAST(sc.revenue AS DECIMAL(23,7))#12] +Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] + +(12) Scan parquet default.store_sales +Output [4]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_sold_date_sk#16 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [4]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +(13) ColumnarToRow [codegen id : 4] +Input [4]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16] -(13) Filter [codegen id : 4] -Input [4]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_store_sk#13) +(14) Filter [codegen id : 4] +Input [4]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16] +Condition : isnotnull(ss_store_sk#14) -(14) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#16] +(15) ReusedExchange [Reuses operator id: 47] +Output [1]: [d_date_sk#17] -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#15] -Right keys [1]: [d_date_sk#16] +(16) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#16] +Right keys [1]: [d_date_sk#17] Join condition: None -(16) Project [codegen id : 4] -Output [3]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14] -Input [5]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#16] - -(17) HashAggregate [codegen id : 4] -Input [3]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14] -Keys [2]: [ss_store_sk#13, ss_item_sk#12] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum#17] -Results [3]: [ss_store_sk#13, ss_item_sk#12, sum#18] +(17) Project [codegen id : 4] +Output [3]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15] +Input [5]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16, d_date_sk#17] -(18) Exchange -Input [3]: [ss_store_sk#13, ss_item_sk#12, sum#18] -Arguments: hashpartitioning(ss_store_sk#13, ss_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#19] +(18) HashAggregate [codegen id : 4] +Input [3]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15] +Keys [2]: [ss_store_sk#14, ss_item_sk#13] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#15))] +Aggregate Attributes [1]: [sum#18] +Results [3]: [ss_store_sk#14, ss_item_sk#13, sum#19] -(19) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#13, ss_item_sk#12, sum#18] -Keys [2]: [ss_store_sk#13, ss_item_sk#12] -Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] -Results [2]: [ss_store_sk#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS revenue#21] +(19) Exchange +Input [3]: [ss_store_sk#14, ss_item_sk#13, sum#19] +Arguments: hashpartitioning(ss_store_sk#14, ss_item_sk#13, 5), ENSURE_REQUIREMENTS, [id=#20] (20) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#13, revenue#21] -Keys [1]: [ss_store_sk#13] -Functions [1]: [partial_avg(revenue#21)] -Aggregate Attributes [2]: [sum#22, count#23] -Results [3]: [ss_store_sk#13, sum#24, count#25] - -(21) Exchange -Input [3]: [ss_store_sk#13, sum#24, count#25] -Arguments: hashpartitioning(ss_store_sk#13, 5), ENSURE_REQUIREMENTS, [id=#26] - -(22) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#13, sum#24, count#25] -Keys [1]: [ss_store_sk#13] -Functions [1]: [avg(revenue#21)] -Aggregate Attributes [1]: [avg(revenue#21)#27] -Results [2]: [ss_store_sk#13, avg(revenue#21)#27 AS ave#28] - -(23) BroadcastExchange -Input [2]: [ss_store_sk#13, ave#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] - -(24) BroadcastHashJoin [codegen id : 8] +Input [3]: [ss_store_sk#14, ss_item_sk#13, sum#19] +Keys [2]: [ss_store_sk#14, ss_item_sk#13] +Functions [1]: [sum(UnscaledValue(ss_sales_price#15))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#15))#21] +Results [2]: [ss_store_sk#14, MakeDecimal(sum(UnscaledValue(ss_sales_price#15))#21,17,2) AS revenue#22] + +(21) HashAggregate [codegen id : 5] +Input [2]: [ss_store_sk#14, revenue#22] +Keys [1]: [ss_store_sk#14] +Functions [1]: [partial_avg(revenue#22)] +Aggregate Attributes [2]: [sum#23, count#24] +Results [3]: [ss_store_sk#14, sum#25, count#26] + +(22) Exchange +Input [3]: [ss_store_sk#14, sum#25, count#26] +Arguments: hashpartitioning(ss_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#27] + +(23) HashAggregate [codegen id : 6] +Input [3]: [ss_store_sk#14, sum#25, count#26] +Keys [1]: [ss_store_sk#14] +Functions [1]: [avg(revenue#22)] +Aggregate Attributes [1]: [avg(revenue#22)#28] +Results [2]: [ss_store_sk#14, CheckOverflow((0.100000 * promote_precision(avg(revenue#22)#28)), DecimalType(23,7), true) AS (0.100000BD * sb.ave)#29] + +(24) BroadcastExchange +Input [2]: [ss_store_sk#14, (0.100000BD * sb.ave)#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] + +(25) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [ss_store_sk#13] -Join condition: (cast(revenue#11 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#28)), DecimalType(23,7), true)) +Right keys [1]: [ss_store_sk#14] +Join condition: (CAST(sc.revenue AS DECIMAL(23,7))#12 <= (0.100000BD * sb.ave)#29) -(25) Project [codegen id : 8] +(26) Project [codegen id : 8] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#11, ss_store_sk#13, ave#28] +Input [6]: [ss_store_sk#2, ss_item_sk#1, revenue#11, CAST(sc.revenue AS DECIMAL(23,7))#12, ss_store_sk#14, (0.100000BD * sb.ave)#29] -(26) Scan parquet default.store -Output [2]: [s_store_sk#30, s_store_name#31] +(27) Scan parquet default.store +Output [2]: [s_store_sk#31, s_store_name#32] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#30, s_store_name#31] +(28) ColumnarToRow [codegen id : 7] +Input [2]: [s_store_sk#31, s_store_name#32] -(28) Filter [codegen id : 7] -Input [2]: [s_store_sk#30, s_store_name#31] -Condition : isnotnull(s_store_sk#30) +(29) Filter [codegen id : 7] +Input [2]: [s_store_sk#31, s_store_name#32] +Condition : isnotnull(s_store_sk#31) -(29) BroadcastExchange -Input [2]: [s_store_sk#30, s_store_name#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] +(30) BroadcastExchange +Input [2]: [s_store_sk#31, s_store_name#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] -(30) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#30] +Right keys [1]: [s_store_sk#31] Join condition: None -(31) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#11, s_store_name#31] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#11, s_store_sk#30, s_store_name#31] +(32) Project [codegen id : 8] +Output [3]: [ss_item_sk#1, revenue#11, s_store_name#32] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#11, s_store_sk#31, s_store_name#32] -(32) Exchange -Input [3]: [ss_item_sk#1, revenue#11, s_store_name#31] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#33] +(33) Exchange +Input [3]: [ss_item_sk#1, revenue#11, s_store_name#32] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#34] -(33) Sort [codegen id : 9] -Input [3]: [ss_item_sk#1, revenue#11, s_store_name#31] +(34) Sort [codegen id : 9] +Input [3]: [ss_item_sk#1, revenue#11, s_store_name#32] Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 -(34) Scan parquet default.item -Output [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] +(35) Scan parquet default.item +Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] -Input [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] +(36) ColumnarToRow [codegen id : 10] +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -(36) Filter [codegen id : 10] -Input [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] -Condition : isnotnull(i_item_sk#34) +(37) Filter [codegen id : 10] +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Condition : isnotnull(i_item_sk#35) -(37) Exchange -Input [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] -Arguments: hashpartitioning(i_item_sk#34, 5), ENSURE_REQUIREMENTS, [id=#39] +(38) Exchange +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Arguments: hashpartitioning(i_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#40] -(38) Sort [codegen id : 11] -Input [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] -Arguments: [i_item_sk#34 ASC NULLS FIRST], false, 0 +(39) Sort [codegen id : 11] +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Arguments: [i_item_sk#35 ASC NULLS FIRST], false, 0 -(39) SortMergeJoin [codegen id : 12] +(40) SortMergeJoin [codegen id : 12] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#34] +Right keys [1]: [i_item_sk#35] Join condition: None -(40) Project [codegen id : 12] -Output [6]: [s_store_name#31, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] -Input [8]: [ss_item_sk#1, revenue#11, s_store_name#31, i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] +(41) Project [codegen id : 12] +Output [6]: [s_store_name#32, i_item_desc#36, revenue#11, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [8]: [ss_item_sk#1, revenue#11, s_store_name#32, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -(41) TakeOrderedAndProject -Input [6]: [s_store_name#31, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] -Arguments: 100, [s_store_name#31 ASC NULLS FIRST, i_item_desc#35 ASC NULLS FIRST], [s_store_name#31, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] +(42) TakeOrderedAndProject +Input [6]: [s_store_name#32, i_item_desc#36, revenue#11, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Arguments: 100, [s_store_name#32 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#32, i_item_desc#36, revenue#11, i_current_price#37, i_wholesale_cost#38, i_brand#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (46) -+- * Project (45) - +- * Filter (44) - +- * ColumnarToRow (43) - +- Scan parquet default.date_dim (42) +BroadcastExchange (47) ++- * Project (46) + +- * Filter (45) + +- * ColumnarToRow (44) + +- Scan parquet default.date_dim (43) -(42) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#40] +(43) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#40] +(44) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#41] -(44) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#40] -Condition : (((isnotnull(d_month_seq#40) AND (d_month_seq#40 >= 1176)) AND (d_month_seq#40 <= 1187)) AND isnotnull(d_date_sk#6)) +(45) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#41] +Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= 1176)) AND (d_month_seq#41 <= 1187)) AND isnotnull(d_date_sk#6)) -(45) Project [codegen id : 1] +(46) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_month_seq#40] +Input [2]: [d_date_sk#6, d_month_seq#41] -(46) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] -Subquery:2 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 12 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt index 08dbe2abd6959..9e6bfde1bf48f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt @@ -11,33 +11,34 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes Project [ss_item_sk,revenue,s_store_name] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk,ss_item_sk,revenue] - BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] - Filter [revenue] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - InputAdapter - Exchange [ss_store_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastHashJoin [ss_store_sk,ss_store_sk,CAST(sc.revenue AS DECIMAL(23,7)),(0.100000BD * sb.ave)] + Project [ss_store_sk,ss_item_sk,revenue] + Filter [revenue] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] + InputAdapter + Exchange [ss_store_sk,ss_item_sk] #2 + WholeStageCodegen (2) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 WholeStageCodegen (6) - HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),(0.100000BD * sb.ave),sum,count] InputAdapter Exchange [ss_store_sk] #5 WholeStageCodegen (5) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index 49cc9f75956a2..7a601935907b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -139,84 +139,84 @@ Right keys [1]: [i_item_sk#15] Join condition: None (22) Project [codegen id : 9] -Output [7]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19] +Output [8]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19, cast(revenue#13 as decimal(23,7)) AS CAST(sc.revenue AS DECIMAL(23,7))#21] Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#13, i_item_sk#15, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19] (23) Scan parquet default.store_sales -Output [4]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24] +Output [4]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [4]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24] +Input [4]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25] (25) Filter [codegen id : 6] -Input [4]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24] -Condition : isnotnull(ss_store_sk#22) +Input [4]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_store_sk#23) (26) ReusedExchange [Reuses operator id: 43] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#26] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#24] -Right keys [1]: [d_date_sk#25] +Left keys [1]: [ss_sold_date_sk#25] +Right keys [1]: [d_date_sk#26] Join condition: None (28) Project [codegen id : 6] -Output [3]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23] -Input [5]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24, d_date_sk#25] +Output [3]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24] +Input [5]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#26] (29) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23] -Keys [2]: [ss_store_sk#22, ss_item_sk#21] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#23))] -Aggregate Attributes [1]: [sum#26] -Results [3]: [ss_store_sk#22, ss_item_sk#21, sum#27] +Input [3]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24] +Keys [2]: [ss_store_sk#23, ss_item_sk#22] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] +Aggregate Attributes [1]: [sum#27] +Results [3]: [ss_store_sk#23, ss_item_sk#22, sum#28] (30) Exchange -Input [3]: [ss_store_sk#22, ss_item_sk#21, sum#27] -Arguments: hashpartitioning(ss_store_sk#22, ss_item_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [ss_store_sk#23, ss_item_sk#22, sum#28] +Arguments: hashpartitioning(ss_store_sk#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [id=#29] (31) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#22, ss_item_sk#21, sum#27] -Keys [2]: [ss_store_sk#22, ss_item_sk#21] -Functions [1]: [sum(UnscaledValue(ss_sales_price#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#23))#29] -Results [2]: [ss_store_sk#22, MakeDecimal(sum(UnscaledValue(ss_sales_price#23))#29,17,2) AS revenue#30] +Input [3]: [ss_store_sk#23, ss_item_sk#22, sum#28] +Keys [2]: [ss_store_sk#23, ss_item_sk#22] +Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#24))#30] +Results [2]: [ss_store_sk#23, MakeDecimal(sum(UnscaledValue(ss_sales_price#24))#30,17,2) AS revenue#31] (32) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#22, revenue#30] -Keys [1]: [ss_store_sk#22] -Functions [1]: [partial_avg(revenue#30)] -Aggregate Attributes [2]: [sum#31, count#32] -Results [3]: [ss_store_sk#22, sum#33, count#34] +Input [2]: [ss_store_sk#23, revenue#31] +Keys [1]: [ss_store_sk#23] +Functions [1]: [partial_avg(revenue#31)] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ss_store_sk#23, sum#34, count#35] (33) Exchange -Input [3]: [ss_store_sk#22, sum#33, count#34] -Arguments: hashpartitioning(ss_store_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [ss_store_sk#23, sum#34, count#35] +Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, [id=#36] (34) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#22, sum#33, count#34] -Keys [1]: [ss_store_sk#22] -Functions [1]: [avg(revenue#30)] -Aggregate Attributes [1]: [avg(revenue#30)#36] -Results [2]: [ss_store_sk#22, avg(revenue#30)#36 AS ave#37] +Input [3]: [ss_store_sk#23, sum#34, count#35] +Keys [1]: [ss_store_sk#23] +Functions [1]: [avg(revenue#31)] +Aggregate Attributes [1]: [avg(revenue#31)#37] +Results [2]: [ss_store_sk#23, CheckOverflow((0.100000 * promote_precision(avg(revenue#31)#37)), DecimalType(23,7), true) AS (0.100000BD * sb.ave)#38] (35) BroadcastExchange -Input [2]: [ss_store_sk#22, ave#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] +Input [2]: [ss_store_sk#23, (0.100000BD * sb.ave)#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] -Right keys [1]: [ss_store_sk#22] -Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#37)), DecimalType(23,7), true)) +Right keys [1]: [ss_store_sk#23] +Join condition: (CAST(sc.revenue AS DECIMAL(23,7))#21 <= (0.100000BD * sb.ave)#38) (37) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19, ss_store_sk#22, ave#37] +Input [10]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19, CAST(sc.revenue AS DECIMAL(23,7))#21, ss_store_sk#23, (0.100000BD * sb.ave)#38] (38) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] @@ -233,27 +233,27 @@ BroadcastExchange (43) (39) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#39] +Output [2]: [d_date_sk#8, d_month_seq#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#8, d_month_seq#39] +Input [2]: [d_date_sk#8, d_month_seq#40] (41) Filter [codegen id : 1] -Input [2]: [d_date_sk#8, d_month_seq#39] -Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1176)) AND (d_month_seq#39 <= 1187)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#8, d_month_seq#40] +Condition : (((isnotnull(d_month_seq#40) AND (d_month_seq#40 >= 1176)) AND (d_month_seq#40 <= 1187)) AND isnotnull(d_date_sk#8)) (42) Project [codegen id : 1] Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_month_seq#39] +Input [2]: [d_date_sk#8, d_month_seq#40] (43) BroadcastExchange Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index 4b519f37a58bf..e407c9cb608b5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] WholeStageCodegen (9) Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] + BroadcastHashJoin [ss_store_sk,ss_store_sk,CAST(sc.revenue AS DECIMAL(23,7)),(0.100000BD * sb.ave)] Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [s_store_name,ss_store_sk,ss_item_sk,revenue] @@ -45,7 +45,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes InputAdapter BroadcastExchange #5 WholeStageCodegen (8) - HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),(0.100000BD * sb.ave),sum,count] InputAdapter Exchange [ss_store_sk] #6 WholeStageCodegen (7) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index d9011c1d57de8..f088b574e82a1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -1,20 +1,20 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * SortMergeJoin LeftOuter (65) - :- * Sort (58) - : +- Exchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * SortMergeJoin Inner (49) - : : :- * Sort (37) - : : : +- Exchange (36) - : : : +- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) +TakeOrderedAndProject (80) ++- * HashAggregate (79) + +- Exchange (78) + +- * HashAggregate (77) + +- * Project (76) + +- * SortMergeJoin LeftOuter (75) + :- * Sort (68) + : +- Exchange (67) + : +- * Project (66) + : +- * BroadcastHashJoin LeftOuter BuildRight (65) + : :- * Project (60) + : : +- * SortMergeJoin Inner (59) + : : :- * Sort (47) + : : : +- Exchange (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) : : : :- * Project (32) : : : : +- * SortMergeJoin Inner (31) : : : : :- * Sort (25) @@ -47,35 +47,45 @@ TakeOrderedAndProject (70) : : : : +- * Filter (28) : : : : +- * ColumnarToRow (27) : : : : +- Scan parquet default.item (26) - : : : +- ReusedExchange (33) - : : +- * Sort (48) - : : +- Exchange (47) - : : +- * Project (46) - : : +- * BroadcastHashJoin Inner BuildRight (45) - : : :- * Filter (40) - : : : +- * ColumnarToRow (39) - : : : +- Scan parquet default.inventory (38) - : : +- BroadcastExchange (44) - : : +- * Filter (43) - : : +- * ColumnarToRow (42) - : : +- Scan parquet default.warehouse (41) - : +- BroadcastExchange (54) - : +- * Filter (53) - : +- * ColumnarToRow (52) - : +- Scan parquet default.promotion (51) - +- * Sort (64) - +- Exchange (63) - +- * Project (62) - +- * Filter (61) - +- * ColumnarToRow (60) - +- Scan parquet default.catalog_returns (59) + : : : +- BroadcastExchange (43) + : : : +- * Project (42) + : : : +- * BroadcastHashJoin Inner BuildLeft (41) + : : : :- BroadcastExchange (37) + : : : : +- * Project (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.date_dim (33) + : : : +- * Filter (40) + : : : +- * ColumnarToRow (39) + : : : +- Scan parquet default.date_dim (38) + : : +- * Sort (58) + : : +- Exchange (57) + : : +- * Project (56) + : : +- * BroadcastHashJoin Inner BuildRight (55) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.inventory (48) + : : +- BroadcastExchange (54) + : : +- * Filter (53) + : : +- * ColumnarToRow (52) + : : +- Scan parquet default.warehouse (51) + : +- BroadcastExchange (64) + : +- * Filter (63) + : +- * ColumnarToRow (62) + : +- Scan parquet default.promotion (61) + +- * Sort (74) + +- Exchange (73) + +- * Project (72) + +- * Filter (71) + +- * ColumnarToRow (70) + +- Scan parquet default.catalog_returns (69) (1) Scan parquet default.catalog_sales Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] ReadSchema: struct @@ -87,367 +97,347 @@ Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_s Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) (4) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#10, hd_buy_potential#11] +Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [hd_demo_sk#10, hd_buy_potential#11] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] -Input [2]: [hd_demo_sk#10, hd_buy_potential#11] -Condition : ((isnotnull(hd_buy_potential#11) AND (hd_buy_potential#11 = >10000 )) AND isnotnull(hd_demo_sk#10)) +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000 )) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] -Output [1]: [hd_demo_sk#10] -Input [2]: [hd_demo_sk#10, hd_buy_potential#11] +Output [1]: [hd_demo_sk#9] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (8) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +Input [1]: [hd_demo_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (9) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#10] +Right keys [1]: [hd_demo_sk#9] Join condition: None (10) Project [codegen id : 4] Output [7]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Input [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, hd_demo_sk#10] +Input [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, hd_demo_sk#9] (11) Scan parquet default.customer_demographics -Output [2]: [cd_demo_sk#13, cd_marital_status#14] +Output [2]: [cd_demo_sk#12, cd_marital_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_marital_status), EqualTo(cd_marital_status,D), IsNotNull(cd_demo_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [2]: [cd_demo_sk#13, cd_marital_status#14] +Input [2]: [cd_demo_sk#12, cd_marital_status#13] (13) Filter [codegen id : 2] -Input [2]: [cd_demo_sk#13, cd_marital_status#14] -Condition : ((isnotnull(cd_marital_status#14) AND (cd_marital_status#14 = D)) AND isnotnull(cd_demo_sk#13)) +Input [2]: [cd_demo_sk#12, cd_marital_status#13] +Condition : ((isnotnull(cd_marital_status#13) AND (cd_marital_status#13 = D)) AND isnotnull(cd_demo_sk#12)) (14) Project [codegen id : 2] -Output [1]: [cd_demo_sk#13] -Input [2]: [cd_demo_sk#13, cd_marital_status#14] +Output [1]: [cd_demo_sk#12] +Input [2]: [cd_demo_sk#12, cd_marital_status#13] (15) BroadcastExchange -Input [1]: [cd_demo_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +Input [1]: [cd_demo_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (16) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#13] +Right keys [1]: [cd_demo_sk#12] Join condition: None (17) Project [codegen id : 4] Output [6]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, cd_demo_sk#13] +Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, cd_demo_sk#12] (18) Scan parquet default.date_dim -Output [2]: [d_date_sk#16, d_date#17] +Output [2]: [d_date_sk#15, d_date#16] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#16, d_date#17] +Input [2]: [d_date_sk#15, d_date#16] (20) Filter [codegen id : 3] -Input [2]: [d_date_sk#16, d_date#17] -Condition : (isnotnull(d_date#17) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#15, d_date#16] +Condition : (isnotnull(d_date#16) AND isnotnull(d_date_sk#15)) (21) BroadcastExchange -Input [2]: [d_date_sk#16, d_date#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#18] +Input [2]: [d_date_sk#15, d_date#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (22) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#15] Join condition: None (23) Project [codegen id : 4] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17] -Input [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date_sk#16, d_date#17] +Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16] +Input [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date_sk#15, d_date#16] (24) Exchange -Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17] -Arguments: hashpartitioning(cs_item_sk#4, 5), ENSURE_REQUIREMENTS, [id=#19] +Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16] +Arguments: hashpartitioning(cs_item_sk#4, 5), ENSURE_REQUIREMENTS, [id=#18] (25) Sort [codegen id : 5] -Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17] +Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16] Arguments: [cs_item_sk#4 ASC NULLS FIRST], false, 0 (26) Scan parquet default.item -Output [2]: [i_item_sk#20, i_item_desc#21] +Output [2]: [i_item_sk#19, i_item_desc#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#20, i_item_desc#21] +Input [2]: [i_item_sk#19, i_item_desc#20] (28) Filter [codegen id : 6] -Input [2]: [i_item_sk#20, i_item_desc#21] -Condition : isnotnull(i_item_sk#20) +Input [2]: [i_item_sk#19, i_item_desc#20] +Condition : isnotnull(i_item_sk#19) (29) Exchange -Input [2]: [i_item_sk#20, i_item_desc#21] -Arguments: hashpartitioning(i_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [2]: [i_item_sk#19, i_item_desc#20] +Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 7] -Input [2]: [i_item_sk#20, i_item_desc#21] -Arguments: [i_item_sk#20 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#19, i_item_desc#20] +Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 (31) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#20] +Right keys [1]: [i_item_sk#19] Join condition: None (32) Project [codegen id : 10] -Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21] -Input [8]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_sk#20, i_item_desc#21] +Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_desc#20] +Input [8]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_sk#19, i_item_desc#20] -(33) ReusedExchange [Reuses operator id: 81] -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] +(33) Scan parquet default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] +ReadSchema: struct + +(34) ColumnarToRow [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] + +(35) Filter [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(34) BroadcastHashJoin [codegen id : 10] +(36) Project [codegen id : 8] +Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] + +(37) BroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] + +(38) Scan parquet default.date_dim +Output [2]: [d_date_sk#27, d_week_seq#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(39) ColumnarToRow +Input [2]: [d_date_sk#27, d_week_seq#28] + +(40) Filter +Input [2]: [d_date_sk#27, d_week_seq#28] +Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) + +(41) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [d_week_seq#24] +Right keys [1]: [d_week_seq#28] +Join condition: None + +(42) Project [codegen id : 9] +Output [4]: [d_date_sk#22, d_week_seq#24, d_date_sk#27, d_date#23 + 5 days AS d1.d_date + INTERVAL '5 days'#29] +Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] + +(43) BroadcastExchange +Input [4]: [d_date_sk#22, d_week_seq#24, d_date_sk#27, d1.d_date + INTERVAL '5 days'#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] + +(44) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#23] -Join condition: (d_date#17 > d_date#24 + 5 days) +Right keys [1]: [d_date_sk#22] +Join condition: (d_date#16 > d1.d_date + INTERVAL '5 days'#29) -(35) Project [codegen id : 10] -Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26] -Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21, d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] +(45) Project [codegen id : 10] +Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_desc#20, d_date_sk#22, d_week_seq#24, d_date_sk#27, d1.d_date + INTERVAL '5 days'#29] -(36) Exchange -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26] -Arguments: hashpartitioning(cs_item_sk#4, d_date_sk#26, 5), ENSURE_REQUIREMENTS, [id=#27] +(46) Exchange +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: hashpartitioning(cs_item_sk#4, d_date_sk#27, 5), ENSURE_REQUIREMENTS, [id=#31] -(37) Sort [codegen id : 11] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26] -Arguments: [cs_item_sk#4 ASC NULLS FIRST, d_date_sk#26 ASC NULLS FIRST], false, 0 +(47) Sort [codegen id : 11] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: [cs_item_sk#4 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.inventory -Output [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31] +(48) Scan parquet default.inventory +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#31), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(inv_date_sk#35), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 13] -Input [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31] +(49) ColumnarToRow [codegen id : 13] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] -(40) Filter [codegen id : 13] -Input [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31] -Condition : ((isnotnull(inv_quantity_on_hand#30) AND isnotnull(inv_item_sk#28)) AND isnotnull(inv_warehouse_sk#29)) +(50) Filter [codegen id : 13] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Condition : ((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) -(41) Scan parquet default.warehouse -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] +(51) Scan parquet default.warehouse +Output [2]: [w_warehouse_sk#36, w_warehouse_name#37] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 12] -Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] +(52) ColumnarToRow [codegen id : 12] +Input [2]: [w_warehouse_sk#36, w_warehouse_name#37] -(43) Filter [codegen id : 12] -Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] -Condition : isnotnull(w_warehouse_sk#32) +(53) Filter [codegen id : 12] +Input [2]: [w_warehouse_sk#36, w_warehouse_name#37] +Condition : isnotnull(w_warehouse_sk#36) -(44) BroadcastExchange -Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +(54) BroadcastExchange +Input [2]: [w_warehouse_sk#36, w_warehouse_name#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] -(45) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [inv_warehouse_sk#29] -Right keys [1]: [w_warehouse_sk#32] +(55) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [inv_warehouse_sk#33] +Right keys [1]: [w_warehouse_sk#36] Join condition: None -(46) Project [codegen id : 13] -Output [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] -Input [6]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_sk#32, w_warehouse_name#33] +(56) Project [codegen id : 13] +Output [4]: [inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] +Input [6]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_sk#36, w_warehouse_name#37] -(47) Exchange -Input [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] -Arguments: hashpartitioning(inv_item_sk#28, inv_date_sk#31, 5), ENSURE_REQUIREMENTS, [id=#35] +(57) Exchange +Input [4]: [inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] +Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#35, 5), ENSURE_REQUIREMENTS, [id=#39] -(48) Sort [codegen id : 14] -Input [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] -Arguments: [inv_item_sk#28 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 +(58) Sort [codegen id : 14] +Input [4]: [inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#35 ASC NULLS FIRST], false, 0 -(49) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#4, d_date_sk#26] -Right keys [2]: [inv_item_sk#28, inv_date_sk#31] -Join condition: (inv_quantity_on_hand#30 < cs_quantity#7) +(59) SortMergeJoin [codegen id : 16] +Left keys [2]: [cs_item_sk#4, d_date_sk#27] +Right keys [2]: [inv_item_sk#32, inv_date_sk#35] +Join condition: (inv_quantity_on_hand#34 < cs_quantity#7) -(50) Project [codegen id : 16] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26, inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] +(60) Project [codegen id : 16] +Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#24] +Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] -(51) Scan parquet default.promotion -Output [1]: [p_promo_sk#36] +(61) Scan parquet default.promotion +Output [1]: [p_promo_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 15] -Input [1]: [p_promo_sk#36] +(62) ColumnarToRow [codegen id : 15] +Input [1]: [p_promo_sk#40] -(53) Filter [codegen id : 15] -Input [1]: [p_promo_sk#36] -Condition : isnotnull(p_promo_sk#36) +(63) Filter [codegen id : 15] +Input [1]: [p_promo_sk#40] +Condition : isnotnull(p_promo_sk#40) -(54) BroadcastExchange -Input [1]: [p_promo_sk#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] +(64) BroadcastExchange +Input [1]: [p_promo_sk#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#41] -(55) BroadcastHashJoin [codegen id : 16] +(65) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#36] +Right keys [1]: [p_promo_sk#40] Join condition: None -(56) Project [codegen id : 16] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25, p_promo_sk#36] +(66) Project [codegen id : 16] +Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#24, p_promo_sk#40] -(57) Exchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#38] +(67) Exchange +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#24] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#42] -(58) Sort [codegen id : 17] -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] +(68) Sort [codegen id : 17] +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#24] Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 -(59) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +(69) Scan parquet default.catalog_returns +Output [3]: [cr_item_sk#43, cr_order_number#44, cr_returned_date_sk#45] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(60) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +(70) ColumnarToRow [codegen id : 18] +Input [3]: [cr_item_sk#43, cr_order_number#44, cr_returned_date_sk#45] -(61) Filter [codegen id : 18] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] -Condition : (isnotnull(cr_item_sk#39) AND isnotnull(cr_order_number#40)) +(71) Filter [codegen id : 18] +Input [3]: [cr_item_sk#43, cr_order_number#44, cr_returned_date_sk#45] +Condition : (isnotnull(cr_item_sk#43) AND isnotnull(cr_order_number#44)) -(62) Project [codegen id : 18] -Output [2]: [cr_item_sk#39, cr_order_number#40] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +(72) Project [codegen id : 18] +Output [2]: [cr_item_sk#43, cr_order_number#44] +Input [3]: [cr_item_sk#43, cr_order_number#44, cr_returned_date_sk#45] -(63) Exchange -Input [2]: [cr_item_sk#39, cr_order_number#40] -Arguments: hashpartitioning(cr_item_sk#39, cr_order_number#40, 5), ENSURE_REQUIREMENTS, [id=#42] +(73) Exchange +Input [2]: [cr_item_sk#43, cr_order_number#44] +Arguments: hashpartitioning(cr_item_sk#43, cr_order_number#44, 5), ENSURE_REQUIREMENTS, [id=#46] -(64) Sort [codegen id : 19] -Input [2]: [cr_item_sk#39, cr_order_number#40] -Arguments: [cr_item_sk#39 ASC NULLS FIRST, cr_order_number#40 ASC NULLS FIRST], false, 0 +(74) Sort [codegen id : 19] +Input [2]: [cr_item_sk#43, cr_order_number#44] +Arguments: [cr_item_sk#43 ASC NULLS FIRST, cr_order_number#44 ASC NULLS FIRST], false, 0 -(65) SortMergeJoin [codegen id : 20] +(75) SortMergeJoin [codegen id : 20] Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#39, cr_order_number#40] +Right keys [2]: [cr_item_sk#43, cr_order_number#44] Join condition: None -(66) Project [codegen id : 20] -Output [3]: [w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25, cr_item_sk#39, cr_order_number#40] +(76) Project [codegen id : 20] +Output [3]: [w_warehouse_name#37, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#24, cr_item_sk#43, cr_order_number#44] -(67) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Keys [3]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25] +(77) HashAggregate [codegen id : 20] +Input [3]: [w_warehouse_name#37, i_item_desc#20, d_week_seq#24] +Keys [3]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#43] -Results [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44] +Aggregate Attributes [1]: [count#47] +Results [4]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24, count#48] -(68) Exchange -Input [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44] -Arguments: hashpartitioning(i_item_desc#21, w_warehouse_name#33, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#45] +(78) Exchange +Input [4]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24, count#48] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#37, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [id=#49] -(69) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44] -Keys [3]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25] +(79) HashAggregate [codegen id : 21] +Input [4]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24, count#48] +Keys [3]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#46] -Results [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count(1)#46 AS no_promo#47, count(1)#46 AS promo#48, count(1)#46 AS total_cnt#49] - -(70) TakeOrderedAndProject -Input [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] -Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (81) -+- * Project (80) - +- * BroadcastHashJoin Inner BuildLeft (79) - :- BroadcastExchange (75) - : +- * Project (74) - : +- * Filter (73) - : +- * ColumnarToRow (72) - : +- Scan parquet default.date_dim (71) - +- * Filter (78) - +- * ColumnarToRow (77) - +- Scan parquet default.date_dim (76) - - -(71) Scan parquet default.date_dim -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(72) ColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] - -(73) Filter [codegen id : 1] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] -Condition : ((((isnotnull(d_year#50) AND (d_year#50 = 1999)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) - -(74) Project [codegen id : 1] -Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] - -(75) BroadcastExchange -Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#51] - -(76) Scan parquet default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(77) ColumnarToRow -Input [2]: [d_date_sk#26, d_week_seq#52] - -(78) Filter -Input [2]: [d_date_sk#26, d_week_seq#52] -Condition : (isnotnull(d_week_seq#52) AND isnotnull(d_date_sk#26)) - -(79) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#25] -Right keys [1]: [d_week_seq#52] -Join condition: None - -(80) Project [codegen id : 2] -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] -Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#52] - -(81) BroadcastExchange -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] +Aggregate Attributes [1]: [count(1)#50] +Results [6]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24, count(1)#50 AS no_promo#51, count(1)#50 AS promo#52, count(1)#50 AS total_cnt#53] +(80) TakeOrderedAndProject +Input [6]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24, no_promo#51, promo#52, total_cnt#53] +Arguments: 100, [total_cnt#53 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#37 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#37, d_week_seq#24, no_promo#51, promo#52, total_cnt#53] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt index e838025a71db8..6f6786c9eee3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Exchange [cs_item_sk,d_date_sk] #3 WholeStageCodegen (10) Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d1.d_date + INTERVAL '5 days'] Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date,i_item_desc] SortMergeJoin [cs_item_sk,i_item_sk] InputAdapter @@ -43,25 +43,8 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (2) - Project [d_date_sk,d_date,d_week_seq,d_date_sk] - BroadcastHashJoin [d_week_seq,d_week_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #7 + BroadcastExchange #5 WholeStageCodegen (1) Project [hd_demo_sk] Filter [hd_buy_potential,hd_demo_sk] @@ -69,7 +52,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #8 + BroadcastExchange #6 WholeStageCodegen (2) Project [cd_demo_sk] Filter [cd_marital_status,cd_demo_sk] @@ -77,7 +60,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - BroadcastExchange #9 + BroadcastExchange #7 WholeStageCodegen (3) Filter [d_date,d_date_sk] ColumnarToRow @@ -87,14 +70,29 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom WholeStageCodegen (7) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #10 + Exchange [i_item_sk] #8 WholeStageCodegen (6) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #5 + BroadcastExchange #9 + WholeStageCodegen (9) + Project [d_date_sk,d_week_seq,d_date_sk,d_date] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter WholeStageCodegen (14) Sort [inv_item_sk,inv_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index 2d0a33d0113b9..21714d4eaad80 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -266,127 +266,127 @@ Right keys [2]: [d_week_seq#31, d_date_sk#30] Join condition: None (44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_date#28, d_week_seq#29] +Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29, d_date#28 + 5 days AS d1.d_date + INTERVAL '5 days'#33] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#16, i_item_desc#19, d_date#28, d_week_seq#29, d_date_sk#30, d_week_seq#31] (45) Scan parquet default.date_dim -Output [2]: [d_date_sk#33, d_date#34] +Output [2]: [d_date_sk#34, d_date#35] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (46) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#33, d_date#34] +Input [2]: [d_date_sk#34, d_date#35] (47) Filter [codegen id : 8] -Input [2]: [d_date_sk#33, d_date#34] -Condition : (isnotnull(d_date#34) AND isnotnull(d_date_sk#33)) +Input [2]: [d_date_sk#34, d_date#35] +Condition : (isnotnull(d_date#35) AND isnotnull(d_date_sk#34)) (48) BroadcastExchange -Input [2]: [d_date_sk#33, d_date#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#35] +Input [2]: [d_date_sk#34, d_date#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] (49) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#33] -Join condition: (d_date#34 > d_date#28 + 5 days) +Right keys [1]: [d_date_sk#34] +Join condition: (d_date#35 > d1.d_date + INTERVAL '5 days'#33) (50) Project [codegen id : 10] Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_date#28, d_week_seq#29, d_date_sk#33, d_date#34] +Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29, d1.d_date + INTERVAL '5 days'#33, d_date_sk#34, d_date#35] (51) Scan parquet default.promotion -Output [1]: [p_promo_sk#36] +Output [1]: [p_promo_sk#37] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#36] +Input [1]: [p_promo_sk#37] (53) Filter [codegen id : 9] -Input [1]: [p_promo_sk#36] -Condition : isnotnull(p_promo_sk#36) +Input [1]: [p_promo_sk#37] +Condition : isnotnull(p_promo_sk#37) (54) BroadcastExchange -Input [1]: [p_promo_sk#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] +Input [1]: [p_promo_sk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] (55) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#36] +Right keys [1]: [p_promo_sk#37] Join condition: None (56) Project [codegen id : 10] Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29, p_promo_sk#36] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29, p_promo_sk#37] (57) Exchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#38] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#39] (58) Sort [codegen id : 11] Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29] Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 (59) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +Output [3]: [cr_item_sk#40, cr_order_number#41, cr_returned_date_sk#42] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct (60) ColumnarToRow [codegen id : 12] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +Input [3]: [cr_item_sk#40, cr_order_number#41, cr_returned_date_sk#42] (61) Filter [codegen id : 12] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] -Condition : (isnotnull(cr_item_sk#39) AND isnotnull(cr_order_number#40)) +Input [3]: [cr_item_sk#40, cr_order_number#41, cr_returned_date_sk#42] +Condition : (isnotnull(cr_item_sk#40) AND isnotnull(cr_order_number#41)) (62) Project [codegen id : 12] -Output [2]: [cr_item_sk#39, cr_order_number#40] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +Output [2]: [cr_item_sk#40, cr_order_number#41] +Input [3]: [cr_item_sk#40, cr_order_number#41, cr_returned_date_sk#42] (63) Exchange -Input [2]: [cr_item_sk#39, cr_order_number#40] -Arguments: hashpartitioning(cr_item_sk#39, cr_order_number#40, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [2]: [cr_item_sk#40, cr_order_number#41] +Arguments: hashpartitioning(cr_item_sk#40, cr_order_number#41, 5), ENSURE_REQUIREMENTS, [id=#43] (64) Sort [codegen id : 13] -Input [2]: [cr_item_sk#39, cr_order_number#40] -Arguments: [cr_item_sk#39 ASC NULLS FIRST, cr_order_number#40 ASC NULLS FIRST], false, 0 +Input [2]: [cr_item_sk#40, cr_order_number#41] +Arguments: [cr_item_sk#40 ASC NULLS FIRST, cr_order_number#41 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 14] Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#39, cr_order_number#40] +Right keys [2]: [cr_item_sk#40, cr_order_number#41] Join condition: None (66) Project [codegen id : 14] Output [3]: [w_warehouse_name#16, i_item_desc#19, d_week_seq#29] -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29, cr_item_sk#39, cr_order_number#40] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29, cr_item_sk#40, cr_order_number#41] (67) HashAggregate [codegen id : 14] Input [3]: [w_warehouse_name#16, i_item_desc#19, d_week_seq#29] Keys [3]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#43] -Results [4]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count#44] +Aggregate Attributes [1]: [count#44] +Results [4]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count#45] (68) Exchange -Input [4]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count#44] -Arguments: hashpartitioning(i_item_desc#19, w_warehouse_name#16, d_week_seq#29, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [4]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count#45] +Arguments: hashpartitioning(i_item_desc#19, w_warehouse_name#16, d_week_seq#29, 5), ENSURE_REQUIREMENTS, [id=#46] (69) HashAggregate [codegen id : 15] -Input [4]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count#44] +Input [4]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count#45] Keys [3]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#46] -Results [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count(1)#46 AS no_promo#47, count(1)#46 AS promo#48, count(1)#46 AS total_cnt#49] +Aggregate Attributes [1]: [count(1)#47] +Results [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count(1)#47 AS no_promo#48, count(1)#47 AS promo#49, count(1)#47 AS total_cnt#50] (70) TakeOrderedAndProject -Input [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49] -Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#19 ASC NULLS FIRST, w_warehouse_name#16 ASC NULLS FIRST, d_week_seq#29 ASC NULLS FIRST], [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49] +Input [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#48, promo#49, total_cnt#50] +Arguments: 100, [total_cnt#50 DESC NULLS LAST, i_item_desc#19 ASC NULLS FIRST, w_warehouse_name#16 ASC NULLS FIRST, d_week_seq#29 ASC NULLS FIRST], [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#48, promo#49, total_cnt#50] ===== Subqueries ===== @@ -399,25 +399,25 @@ BroadcastExchange (75) (71) Scan parquet default.date_dim -Output [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#50] +Output [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#51] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (72) ColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#50] +Input [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#51] (73) Filter [codegen id : 1] -Input [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#50] -Condition : ((((isnotnull(d_year#50) AND (d_year#50 = 1999)) AND isnotnull(d_date_sk#27)) AND isnotnull(d_week_seq#29)) AND isnotnull(d_date#28)) +Input [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#51] +Condition : ((((isnotnull(d_year#51) AND (d_year#51 = 1999)) AND isnotnull(d_date_sk#27)) AND isnotnull(d_week_seq#29)) AND isnotnull(d_date#28)) (74) Project [codegen id : 1] Output [3]: [d_date_sk#27, d_date#28, d_week_seq#29] -Input [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#50] +Input [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#51] (75) BroadcastExchange Input [3]: [d_date_sk#27, d_date#28, d_week_seq#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt index 7968b1cb84729..e684dba8c697c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d1.d_date + INTERVAL '5 days'] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,d_date] BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt index e8c77e7de05eb..ce30b7c323c39 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * Project (45) - +- * SortMergeJoin Inner (44) +TakeOrderedAndProject (50) ++- * HashAggregate (49) + +- Exchange (48) + +- * HashAggregate (47) + +- * Project (46) + +- * SortMergeJoin Inner (45) :- * Sort (14) : +- Exchange (13) : +- * Project (12) @@ -19,35 +19,36 @@ TakeOrderedAndProject (49) : +- * Filter (9) : +- * ColumnarToRow (8) : +- Scan parquet default.store (7) - +- * Sort (43) - +- Exchange (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin LeftSemi BuildRight (37) - :- * Filter (17) - : +- * ColumnarToRow (16) - : +- Scan parquet default.customer_address (15) - +- BroadcastExchange (36) - +- * Project (35) - +- * Filter (34) - +- * HashAggregate (33) - +- Exchange (32) - +- * HashAggregate (31) - +- * Project (30) - +- * SortMergeJoin Inner (29) - :- * Sort (22) - : +- Exchange (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.customer_address (18) - +- * Sort (28) - +- Exchange (27) - +- * Project (26) - +- * Filter (25) - +- * ColumnarToRow (24) - +- Scan parquet default.customer (23) + +- * Sort (44) + +- Exchange (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin LeftSemi BuildRight (38) + :- * Project (18) + : +- * Filter (17) + : +- * ColumnarToRow (16) + : +- Scan parquet default.customer_address (15) + +- BroadcastExchange (37) + +- * Project (36) + +- * Filter (35) + +- * HashAggregate (34) + +- Exchange (33) + +- * HashAggregate (32) + +- * Project (31) + +- * SortMergeJoin Inner (30) + :- * Sort (23) + : +- Exchange (22) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.customer_address (19) + +- * Sort (29) + +- Exchange (28) + +- * Project (27) + +- * Filter (26) + +- * ColumnarToRow (25) + +- Scan parquet default.customer (24) (1) Scan parquet default.store_sales @@ -65,7 +66,7 @@ Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(4) ReusedExchange [Reuses operator id: 54] +(4) ReusedExchange [Reuses operator id: 55] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 3] @@ -101,213 +102,217 @@ Right keys [1]: [s_store_sk#6] Join condition: None (12) Project [codegen id : 3] -Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] +Output [3]: [ss_net_profit#2, s_store_name#7, substr(s_zip#8, 1, 2) AS substr(spark_catalog.default.store.s_zip, 1, 2)#10] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#8] (13) Exchange -Input [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] -Arguments: hashpartitioning(substr(s_zip#8, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#10] +Input [3]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10] +Arguments: hashpartitioning(substr(spark_catalog.default.store.s_zip, 1, 2)#10, 5), ENSURE_REQUIREMENTS, [id=#11] (14) Sort [codegen id : 4] -Input [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] -Arguments: [substr(s_zip#8, 1, 2) ASC NULLS FIRST], false, 0 +Input [3]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10] +Arguments: [substr(spark_catalog.default.store.s_zip, 1, 2)#10 ASC NULLS FIRST], false, 0 (15) Scan parquet default.customer_address -Output [1]: [ca_zip#11] +Output [1]: [ca_zip#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] ReadSchema: struct (16) ColumnarToRow [codegen id : 11] -Input [1]: [ca_zip#11] +Input [1]: [ca_zip#12] (17) Filter [codegen id : 11] -Input [1]: [ca_zip#11] -Condition : (substr(ca_zip#11, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#11, 1, 5))) +Input [1]: [ca_zip#12] +Condition : (substr(ca_zip#12, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#12, 1, 5))) -(18) Scan parquet default.customer_address -Output [2]: [ca_address_sk#12, ca_zip#13] +(18) Project [codegen id : 11] +Output [2]: [ca_zip#12, substr(ca_zip#12, 1, 5) AS substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#13] +Input [1]: [ca_zip#12] + +(19) Scan parquet default.customer_address +Output [2]: [ca_address_sk#14, ca_zip#15] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#12, ca_zip#13] +(20) ColumnarToRow [codegen id : 5] +Input [2]: [ca_address_sk#14, ca_zip#15] -(20) Filter [codegen id : 5] -Input [2]: [ca_address_sk#12, ca_zip#13] -Condition : isnotnull(ca_address_sk#12) +(21) Filter [codegen id : 5] +Input [2]: [ca_address_sk#14, ca_zip#15] +Condition : isnotnull(ca_address_sk#14) -(21) Exchange -Input [2]: [ca_address_sk#12, ca_zip#13] -Arguments: hashpartitioning(ca_address_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] +(22) Exchange +Input [2]: [ca_address_sk#14, ca_zip#15] +Arguments: hashpartitioning(ca_address_sk#14, 5), ENSURE_REQUIREMENTS, [id=#16] -(22) Sort [codegen id : 6] -Input [2]: [ca_address_sk#12, ca_zip#13] -Arguments: [ca_address_sk#12 ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 6] +Input [2]: [ca_address_sk#14, ca_zip#15] +Arguments: [ca_address_sk#14 ASC NULLS FIRST], false, 0 -(23) Scan parquet default.customer -Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +(24) Scan parquet default.customer +Output [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 7] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +(25) ColumnarToRow [codegen id : 7] +Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -(25) Filter [codegen id : 7] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Condition : ((isnotnull(c_preferred_cust_flag#16) AND (c_preferred_cust_flag#16 = Y)) AND isnotnull(c_current_addr_sk#15)) +(26) Filter [codegen id : 7] +Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +Condition : ((isnotnull(c_preferred_cust_flag#18) AND (c_preferred_cust_flag#18 = Y)) AND isnotnull(c_current_addr_sk#17)) -(26) Project [codegen id : 7] -Output [1]: [c_current_addr_sk#15] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +(27) Project [codegen id : 7] +Output [1]: [c_current_addr_sk#17] +Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -(27) Exchange -Input [1]: [c_current_addr_sk#15] -Arguments: hashpartitioning(c_current_addr_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] +(28) Exchange +Input [1]: [c_current_addr_sk#17] +Arguments: hashpartitioning(c_current_addr_sk#17, 5), ENSURE_REQUIREMENTS, [id=#19] -(28) Sort [codegen id : 8] -Input [1]: [c_current_addr_sk#15] -Arguments: [c_current_addr_sk#15 ASC NULLS FIRST], false, 0 +(29) Sort [codegen id : 8] +Input [1]: [c_current_addr_sk#17] +Arguments: [c_current_addr_sk#17 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin [codegen id : 9] -Left keys [1]: [ca_address_sk#12] -Right keys [1]: [c_current_addr_sk#15] +(30) SortMergeJoin [codegen id : 9] +Left keys [1]: [ca_address_sk#14] +Right keys [1]: [c_current_addr_sk#17] Join condition: None -(30) Project [codegen id : 9] -Output [1]: [ca_zip#13] -Input [3]: [ca_address_sk#12, ca_zip#13, c_current_addr_sk#15] +(31) Project [codegen id : 9] +Output [1]: [ca_zip#15] +Input [3]: [ca_address_sk#14, ca_zip#15, c_current_addr_sk#17] -(31) HashAggregate [codegen id : 9] -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] +(32) HashAggregate [codegen id : 9] +Input [1]: [ca_zip#15] +Keys [1]: [ca_zip#15] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [2]: [ca_zip#13, count#19] +Aggregate Attributes [1]: [count#20] +Results [2]: [ca_zip#15, count#21] -(32) Exchange -Input [2]: [ca_zip#13, count#19] -Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, [id=#20] +(33) Exchange +Input [2]: [ca_zip#15, count#21] +Arguments: hashpartitioning(ca_zip#15, 5), ENSURE_REQUIREMENTS, [id=#22] -(33) HashAggregate [codegen id : 10] -Input [2]: [ca_zip#13, count#19] -Keys [1]: [ca_zip#13] +(34) HashAggregate [codegen id : 10] +Input [2]: [ca_zip#15, count#21] +Keys [1]: [ca_zip#15] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#21] -Results [2]: [substr(ca_zip#13, 1, 5) AS ca_zip#22, count(1)#21 AS cnt#23] +Aggregate Attributes [1]: [count(1)#23] +Results [2]: [substr(ca_zip#15, 1, 5) AS ca_zip#24, count(1)#23 AS cnt#25] -(34) Filter [codegen id : 10] -Input [2]: [ca_zip#22, cnt#23] -Condition : (cnt#23 > 10) +(35) Filter [codegen id : 10] +Input [2]: [ca_zip#24, cnt#25] +Condition : (cnt#25 > 10) -(35) Project [codegen id : 10] -Output [1]: [ca_zip#22] -Input [2]: [ca_zip#22, cnt#23] +(36) Project [codegen id : 10] +Output [1]: [ca_zip#24] +Input [2]: [ca_zip#24, cnt#25] -(36) BroadcastExchange -Input [1]: [ca_zip#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#24] +(37) BroadcastExchange +Input [1]: [ca_zip#24] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#26] -(37) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [coalesce(substr(ca_zip#11, 1, 5), ), isnull(substr(ca_zip#11, 1, 5))] -Right keys [2]: [coalesce(ca_zip#22, ), isnull(ca_zip#22)] +(38) BroadcastHashJoin [codegen id : 11] +Left keys [2]: [coalesce(substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#13, ), isnull(substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#13)] +Right keys [2]: [coalesce(ca_zip#24, ), isnull(ca_zip#24)] Join condition: None -(38) Project [codegen id : 11] -Output [1]: [substr(ca_zip#11, 1, 5) AS ca_zip#25] -Input [1]: [ca_zip#11] +(39) Project [codegen id : 11] +Output [1]: [substr(ca_zip#12, 1, 5) AS ca_zip#27] +Input [2]: [ca_zip#12, substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#13] -(39) HashAggregate [codegen id : 11] -Input [1]: [ca_zip#25] -Keys [1]: [ca_zip#25] +(40) HashAggregate [codegen id : 11] +Input [1]: [ca_zip#27] +Keys [1]: [ca_zip#27] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#25] +Results [1]: [ca_zip#27] -(40) Exchange -Input [1]: [ca_zip#25] -Arguments: hashpartitioning(ca_zip#25, 5), ENSURE_REQUIREMENTS, [id=#26] +(41) Exchange +Input [1]: [ca_zip#27] +Arguments: hashpartitioning(ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#28] -(41) HashAggregate [codegen id : 12] -Input [1]: [ca_zip#25] -Keys [1]: [ca_zip#25] +(42) HashAggregate [codegen id : 12] +Input [1]: [ca_zip#27] +Keys [1]: [ca_zip#27] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#25] +Results [1]: [substr(ca_zip#27, 1, 2) AS substr(V1.ca_zip, 1, 2)#29] -(42) Exchange -Input [1]: [ca_zip#25] -Arguments: hashpartitioning(substr(ca_zip#25, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#27] +(43) Exchange +Input [1]: [substr(V1.ca_zip, 1, 2)#29] +Arguments: hashpartitioning(substr(V1.ca_zip, 1, 2)#29, 5), ENSURE_REQUIREMENTS, [id=#30] -(43) Sort [codegen id : 13] -Input [1]: [ca_zip#25] -Arguments: [substr(ca_zip#25, 1, 2) ASC NULLS FIRST], false, 0 +(44) Sort [codegen id : 13] +Input [1]: [substr(V1.ca_zip, 1, 2)#29] +Arguments: [substr(V1.ca_zip, 1, 2)#29 ASC NULLS FIRST], false, 0 -(44) SortMergeJoin [codegen id : 14] -Left keys [1]: [substr(s_zip#8, 1, 2)] -Right keys [1]: [substr(ca_zip#25, 1, 2)] +(45) SortMergeJoin [codegen id : 14] +Left keys [1]: [substr(spark_catalog.default.store.s_zip, 1, 2)#10] +Right keys [1]: [substr(V1.ca_zip, 1, 2)#29] Join condition: None -(45) Project [codegen id : 14] +(46) Project [codegen id : 14] Output [2]: [ss_net_profit#2, s_store_name#7] -Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#8, ca_zip#25] +Input [4]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10, substr(V1.ca_zip, 1, 2)#29] -(46) HashAggregate [codegen id : 14] +(47) HashAggregate [codegen id : 14] Input [2]: [ss_net_profit#2, s_store_name#7] Keys [1]: [s_store_name#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [s_store_name#7, sum#29] +Aggregate Attributes [1]: [sum#31] +Results [2]: [s_store_name#7, sum#32] -(47) Exchange -Input [2]: [s_store_name#7, sum#29] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#30] +(48) Exchange +Input [2]: [s_store_name#7, sum#32] +Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#33] -(48) HashAggregate [codegen id : 15] -Input [2]: [s_store_name#7, sum#29] +(49) HashAggregate [codegen id : 15] +Input [2]: [s_store_name#7, sum#32] Keys [1]: [s_store_name#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] -Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS sum(ss_net_profit)#32] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] +Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS sum(ss_net_profit)#35] -(49) TakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#32] -Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#32] +(50) TakeOrderedAndProject +Input [2]: [s_store_name#7, sum(ss_net_profit)#35] +Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (54) -+- * Project (53) - +- * Filter (52) - +- * ColumnarToRow (51) - +- Scan parquet default.date_dim (50) +BroadcastExchange (55) ++- * Project (54) + +- * Filter (53) + +- * ColumnarToRow (52) + +- Scan parquet default.date_dim (51) -(50) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#33, d_qoy#34] +(51) Scan parquet default.date_dim +Output [3]: [d_date_sk#5, d_year#36, d_qoy#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(51) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#33, d_qoy#34] +(52) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#5, d_year#36, d_qoy#37] -(52) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#33, d_qoy#34] -Condition : ((((isnotnull(d_qoy#34) AND isnotnull(d_year#33)) AND (d_qoy#34 = 2)) AND (d_year#33 = 1998)) AND isnotnull(d_date_sk#5)) +(53) Filter [codegen id : 1] +Input [3]: [d_date_sk#5, d_year#36, d_qoy#37] +Condition : ((((isnotnull(d_qoy#37) AND isnotnull(d_year#36)) AND (d_qoy#37 = 2)) AND (d_year#36 = 1998)) AND isnotnull(d_date_sk#5)) -(53) Project [codegen id : 1] +(54) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#33, d_qoy#34] +Input [3]: [d_date_sk#5, d_year#36, d_qoy#37] -(54) BroadcastExchange +(55) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt index 84ac2edd606cf..aab384d8e6512 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt @@ -6,12 +6,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (14) HashAggregate [s_store_name,ss_net_profit] [sum,sum] Project [ss_net_profit,s_store_name] - SortMergeJoin [s_zip,ca_zip] + SortMergeJoin [substr(spark_catalog.default.store.s_zip, 1, 2),substr(V1.ca_zip, 1, 2)] InputAdapter WholeStageCodegen (4) - Sort [s_zip] + Sort [substr(spark_catalog.default.store.s_zip, 1, 2)] InputAdapter - Exchange [s_zip] #2 + Exchange [substr(spark_catalog.default.store.s_zip, 1, 2)] #2 WholeStageCodegen (3) Project [ss_net_profit,s_store_name,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -40,21 +40,22 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] Scan parquet default.store [s_store_sk,s_store_name,s_zip] InputAdapter WholeStageCodegen (13) - Sort [ca_zip] + Sort [substr(V1.ca_zip, 1, 2)] InputAdapter - Exchange [ca_zip] #5 + Exchange [substr(V1.ca_zip, 1, 2)] #5 WholeStageCodegen (12) - HashAggregate [ca_zip] + HashAggregate [ca_zip] [substr(V1.ca_zip, 1, 2)] InputAdapter Exchange [ca_zip] #6 WholeStageCodegen (11) HashAggregate [ca_zip] Project [ca_zip] - BroadcastHashJoin [ca_zip,ca_zip] - Filter [ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_zip] + BroadcastHashJoin [substr(spark_catalog.default.customer_address.ca_zip, 1, 5),ca_zip] + Project [ca_zip] + Filter [ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (10) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index efac82f31fdb4..08f1b447670fe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (12) : +- * BroadcastHashJoin Inner BuildRight (11) : :- * Project (6) @@ -17,31 +17,32 @@ TakeOrderedAndProject (43) : +- * Filter (9) : +- * ColumnarToRow (8) : +- Scan parquet default.store (7) - +- BroadcastExchange (37) - +- * HashAggregate (36) - +- Exchange (35) - +- * HashAggregate (34) - +- * Project (33) - +- * BroadcastHashJoin LeftSemi BuildRight (32) - :- * Filter (15) - : +- * ColumnarToRow (14) - : +- Scan parquet default.customer_address (13) - +- BroadcastExchange (31) - +- * Project (30) - +- * Filter (29) - +- * HashAggregate (28) - +- Exchange (27) - +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Filter (18) - : +- * ColumnarToRow (17) - : +- Scan parquet default.customer_address (16) - +- BroadcastExchange (23) - +- * Project (22) - +- * Filter (21) - +- * ColumnarToRow (20) - +- Scan parquet default.customer (19) + +- BroadcastExchange (38) + +- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- * Project (34) + +- * BroadcastHashJoin LeftSemi BuildRight (33) + :- * Project (16) + : +- * Filter (15) + : +- * ColumnarToRow (14) + : +- Scan parquet default.customer_address (13) + +- BroadcastExchange (32) + +- * Project (31) + +- * Filter (30) + +- * HashAggregate (29) + +- Exchange (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * Filter (19) + : +- * ColumnarToRow (18) + : +- Scan parquet default.customer_address (17) + +- BroadcastExchange (24) + +- * Project (23) + +- * Filter (22) + +- * ColumnarToRow (21) + +- Scan parquet default.customer (20) (1) Scan parquet default.store_sales @@ -59,7 +60,7 @@ Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(4) ReusedExchange [Reuses operator id: 48] +(4) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 8] @@ -95,189 +96,193 @@ Right keys [1]: [s_store_sk#6] Join condition: None (12) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] +Output [3]: [ss_net_profit#2, s_store_name#7, substr(s_zip#8, 1, 2) AS substr(spark_catalog.default.store.s_zip, 1, 2)#10] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#8] (13) Scan parquet default.customer_address -Output [1]: [ca_zip#10] +Output [1]: [ca_zip#11] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] ReadSchema: struct (14) ColumnarToRow [codegen id : 6] -Input [1]: [ca_zip#10] +Input [1]: [ca_zip#11] (15) Filter [codegen id : 6] -Input [1]: [ca_zip#10] -Condition : (substr(ca_zip#10, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#10, 1, 5))) +Input [1]: [ca_zip#11] +Condition : (substr(ca_zip#11, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#11, 1, 5))) -(16) Scan parquet default.customer_address -Output [2]: [ca_address_sk#11, ca_zip#12] +(16) Project [codegen id : 6] +Output [2]: [ca_zip#11, substr(ca_zip#11, 1, 5) AS substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#12] +Input [1]: [ca_zip#11] + +(17) Scan parquet default.customer_address +Output [2]: [ca_address_sk#13, ca_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#11, ca_zip#12] +(18) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#13, ca_zip#14] -(18) Filter [codegen id : 4] -Input [2]: [ca_address_sk#11, ca_zip#12] -Condition : isnotnull(ca_address_sk#11) +(19) Filter [codegen id : 4] +Input [2]: [ca_address_sk#13, ca_zip#14] +Condition : isnotnull(ca_address_sk#13) -(19) Scan parquet default.customer -Output [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] +(20) Scan parquet default.customer +Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 3] -Input [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] +(21) ColumnarToRow [codegen id : 3] +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -(21) Filter [codegen id : 3] -Input [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] -Condition : ((isnotnull(c_preferred_cust_flag#14) AND (c_preferred_cust_flag#14 = Y)) AND isnotnull(c_current_addr_sk#13)) +(22) Filter [codegen id : 3] +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Condition : ((isnotnull(c_preferred_cust_flag#16) AND (c_preferred_cust_flag#16 = Y)) AND isnotnull(c_current_addr_sk#15)) -(22) Project [codegen id : 3] -Output [1]: [c_current_addr_sk#13] -Input [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] +(23) Project [codegen id : 3] +Output [1]: [c_current_addr_sk#15] +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -(23) BroadcastExchange -Input [1]: [c_current_addr_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +(24) BroadcastExchange +Input [1]: [c_current_addr_sk#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ca_address_sk#11] -Right keys [1]: [c_current_addr_sk#13] +(25) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ca_address_sk#13] +Right keys [1]: [c_current_addr_sk#15] Join condition: None -(25) Project [codegen id : 4] -Output [1]: [ca_zip#12] -Input [3]: [ca_address_sk#11, ca_zip#12, c_current_addr_sk#13] +(26) Project [codegen id : 4] +Output [1]: [ca_zip#14] +Input [3]: [ca_address_sk#13, ca_zip#14, c_current_addr_sk#15] -(26) HashAggregate [codegen id : 4] -Input [1]: [ca_zip#12] -Keys [1]: [ca_zip#12] +(27) HashAggregate [codegen id : 4] +Input [1]: [ca_zip#14] +Keys [1]: [ca_zip#14] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#16] -Results [2]: [ca_zip#12, count#17] +Aggregate Attributes [1]: [count#18] +Results [2]: [ca_zip#14, count#19] -(27) Exchange -Input [2]: [ca_zip#12, count#17] -Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [id=#18] +(28) Exchange +Input [2]: [ca_zip#14, count#19] +Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [id=#20] -(28) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#12, count#17] -Keys [1]: [ca_zip#12] +(29) HashAggregate [codegen id : 5] +Input [2]: [ca_zip#14, count#19] +Keys [1]: [ca_zip#14] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#19] -Results [2]: [substr(ca_zip#12, 1, 5) AS ca_zip#20, count(1)#19 AS cnt#21] +Aggregate Attributes [1]: [count(1)#21] +Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#22, count(1)#21 AS cnt#23] -(29) Filter [codegen id : 5] -Input [2]: [ca_zip#20, cnt#21] -Condition : (cnt#21 > 10) +(30) Filter [codegen id : 5] +Input [2]: [ca_zip#22, cnt#23] +Condition : (cnt#23 > 10) -(30) Project [codegen id : 5] -Output [1]: [ca_zip#20] -Input [2]: [ca_zip#20, cnt#21] +(31) Project [codegen id : 5] +Output [1]: [ca_zip#22] +Input [2]: [ca_zip#22, cnt#23] -(31) BroadcastExchange -Input [1]: [ca_zip#20] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#22] +(32) BroadcastExchange +Input [1]: [ca_zip#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#24] -(32) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [coalesce(substr(ca_zip#10, 1, 5), ), isnull(substr(ca_zip#10, 1, 5))] -Right keys [2]: [coalesce(ca_zip#20, ), isnull(ca_zip#20)] +(33) BroadcastHashJoin [codegen id : 6] +Left keys [2]: [coalesce(substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#12, ), isnull(substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#12)] +Right keys [2]: [coalesce(ca_zip#22, ), isnull(ca_zip#22)] Join condition: None -(33) Project [codegen id : 6] -Output [1]: [substr(ca_zip#10, 1, 5) AS ca_zip#23] -Input [1]: [ca_zip#10] +(34) Project [codegen id : 6] +Output [1]: [substr(ca_zip#11, 1, 5) AS ca_zip#25] +Input [2]: [ca_zip#11, substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#12] -(34) HashAggregate [codegen id : 6] -Input [1]: [ca_zip#23] -Keys [1]: [ca_zip#23] +(35) HashAggregate [codegen id : 6] +Input [1]: [ca_zip#25] +Keys [1]: [ca_zip#25] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#23] +Results [1]: [ca_zip#25] -(35) Exchange -Input [1]: [ca_zip#23] -Arguments: hashpartitioning(ca_zip#23, 5), ENSURE_REQUIREMENTS, [id=#24] +(36) Exchange +Input [1]: [ca_zip#25] +Arguments: hashpartitioning(ca_zip#25, 5), ENSURE_REQUIREMENTS, [id=#26] -(36) HashAggregate [codegen id : 7] -Input [1]: [ca_zip#23] -Keys [1]: [ca_zip#23] +(37) HashAggregate [codegen id : 7] +Input [1]: [ca_zip#25] +Keys [1]: [ca_zip#25] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#23] +Results [1]: [substr(ca_zip#25, 1, 2) AS substr(V1.ca_zip, 1, 2)#27] -(37) BroadcastExchange -Input [1]: [ca_zip#23] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [id=#25] +(38) BroadcastExchange +Input [1]: [substr(V1.ca_zip, 1, 2)#27] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#28] -(38) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [substr(s_zip#8, 1, 2)] -Right keys [1]: [substr(ca_zip#23, 1, 2)] +(39) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [substr(spark_catalog.default.store.s_zip, 1, 2)#10] +Right keys [1]: [substr(V1.ca_zip, 1, 2)#27] Join condition: None -(39) Project [codegen id : 8] +(40) Project [codegen id : 8] Output [2]: [ss_net_profit#2, s_store_name#7] -Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#8, ca_zip#23] +Input [4]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10, substr(V1.ca_zip, 1, 2)#27] -(40) HashAggregate [codegen id : 8] +(41) HashAggregate [codegen id : 8] Input [2]: [ss_net_profit#2, s_store_name#7] Keys [1]: [s_store_name#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#26] -Results [2]: [s_store_name#7, sum#27] +Aggregate Attributes [1]: [sum#29] +Results [2]: [s_store_name#7, sum#30] -(41) Exchange -Input [2]: [s_store_name#7, sum#27] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#28] +(42) Exchange +Input [2]: [s_store_name#7, sum#30] +Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#31] -(42) HashAggregate [codegen id : 9] -Input [2]: [s_store_name#7, sum#27] +(43) HashAggregate [codegen id : 9] +Input [2]: [s_store_name#7, sum#30] Keys [1]: [s_store_name#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#29] -Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#29,17,2) AS sum(ss_net_profit)#30] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#32] +Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#32,17,2) AS sum(ss_net_profit)#33] -(43) TakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#30] -Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#30] +(44) TakeOrderedAndProject +Input [2]: [s_store_name#7, sum(ss_net_profit)#33] +Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#33] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (48) -+- * Project (47) - +- * Filter (46) - +- * ColumnarToRow (45) - +- Scan parquet default.date_dim (44) +BroadcastExchange (49) ++- * Project (48) + +- * Filter (47) + +- * ColumnarToRow (46) + +- Scan parquet default.date_dim (45) -(44) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#31, d_qoy#32] +(45) Scan parquet default.date_dim +Output [3]: [d_date_sk#5, d_year#34, d_qoy#35] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#31, d_qoy#32] +(46) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#5, d_year#34, d_qoy#35] -(46) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#31, d_qoy#32] -Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 2)) AND (d_year#31 = 1998)) AND isnotnull(d_date_sk#5)) +(47) Filter [codegen id : 1] +Input [3]: [d_date_sk#5, d_year#34, d_qoy#35] +Condition : ((((isnotnull(d_qoy#35) AND isnotnull(d_year#34)) AND (d_qoy#35 = 2)) AND (d_year#34 = 1998)) AND isnotnull(d_date_sk#5)) -(47) Project [codegen id : 1] +(48) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#31, d_qoy#32] +Input [3]: [d_date_sk#5, d_year#34, d_qoy#35] -(48) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index a28dcb87ee2e4..94ac2b90c3201 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (8) HashAggregate [s_store_name,ss_net_profit] [sum,sum] Project [ss_net_profit,s_store_name] - BroadcastHashJoin [s_zip,ca_zip] + BroadcastHashJoin [substr(spark_catalog.default.store.s_zip, 1, 2),substr(V1.ca_zip, 1, 2)] Project [ss_net_profit,s_store_name,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk,ss_net_profit] @@ -35,17 +35,18 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) - HashAggregate [ca_zip] + HashAggregate [ca_zip] [substr(V1.ca_zip, 1, 2)] InputAdapter Exchange [ca_zip] #5 WholeStageCodegen (6) HashAggregate [ca_zip] Project [ca_zip] - BroadcastHashJoin [ca_zip,ca_zip] - Filter [ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_zip] + BroadcastHashJoin [substr(spark_catalog.default.customer_address.ca_zip, 1, 5),ca_zip] + Project [ca_zip] + Filter [ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt index 84c2fd54d5f4b..130a57b39e1b7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt @@ -209,112 +209,112 @@ Right keys [1]: [ctr_customer_sk#35] Join condition: None (34) Project [codegen id : 20] -Output [17]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#36, ctr_total_return#37] +Output [18]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#36, ctr_total_return#37, cast(ctr_total_return#37 as decimal(24,7)) AS CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#39] Input [19]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_customer_sk#35, ctr_state#36, ctr_total_return#37] (35) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, cr_returned_date_sk#42] +Output [4]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42, cr_returned_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#42), dynamicpruningexpression(cr_returned_date_sk#42 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cr_returned_date_sk#43), dynamicpruningexpression(cr_returned_date_sk#43 IN dynamicpruning#25)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 13] -Input [4]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, cr_returned_date_sk#42] +Input [4]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42, cr_returned_date_sk#43] (37) Filter [codegen id : 13] -Input [4]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, cr_returned_date_sk#42] -Condition : isnotnull(cr_returning_addr_sk#40) +Input [4]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42, cr_returned_date_sk#43] +Condition : isnotnull(cr_returning_addr_sk#41) (38) ReusedExchange [Reuses operator id: 62] -Output [1]: [d_date_sk#43] +Output [1]: [d_date_sk#44] (39) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#42] -Right keys [1]: [d_date_sk#43] +Left keys [1]: [cr_returned_date_sk#43] +Right keys [1]: [d_date_sk#44] Join condition: None (40) Project [codegen id : 13] -Output [3]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41] -Input [5]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, cr_returned_date_sk#42, d_date_sk#43] +Output [3]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42] +Input [5]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42, cr_returned_date_sk#43, d_date_sk#44] (41) Exchange -Input [3]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41] -Arguments: hashpartitioning(cr_returning_addr_sk#40, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42] +Arguments: hashpartitioning(cr_returning_addr_sk#41, 5), ENSURE_REQUIREMENTS, [id=#45] (42) Sort [codegen id : 14] -Input [3]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41] -Arguments: [cr_returning_addr_sk#40 ASC NULLS FIRST], false, 0 +Input [3]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42] +Arguments: [cr_returning_addr_sk#41 ASC NULLS FIRST], false, 0 (43) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#45, ca_state#46] +Output [2]: [ca_address_sk#46, ca_state#47] (44) Sort [codegen id : 16] -Input [2]: [ca_address_sk#45, ca_state#46] -Arguments: [ca_address_sk#45 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#46, ca_state#47] +Arguments: [ca_address_sk#46 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] -Left keys [1]: [cr_returning_addr_sk#40] -Right keys [1]: [ca_address_sk#45] +Left keys [1]: [cr_returning_addr_sk#41] +Right keys [1]: [ca_address_sk#46] Join condition: None (46) Project [codegen id : 17] -Output [3]: [cr_returning_customer_sk#39, cr_return_amt_inc_tax#41, ca_state#46] -Input [5]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, ca_address_sk#45, ca_state#46] +Output [3]: [cr_returning_customer_sk#40, cr_return_amt_inc_tax#42, ca_state#47] +Input [5]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42, ca_address_sk#46, ca_state#47] (47) HashAggregate [codegen id : 17] -Input [3]: [cr_returning_customer_sk#39, cr_return_amt_inc_tax#41, ca_state#46] -Keys [2]: [cr_returning_customer_sk#39, ca_state#46] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#41))] -Aggregate Attributes [1]: [sum#47] -Results [3]: [cr_returning_customer_sk#39, ca_state#46, sum#48] +Input [3]: [cr_returning_customer_sk#40, cr_return_amt_inc_tax#42, ca_state#47] +Keys [2]: [cr_returning_customer_sk#40, ca_state#47] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#42))] +Aggregate Attributes [1]: [sum#48] +Results [3]: [cr_returning_customer_sk#40, ca_state#47, sum#49] (48) Exchange -Input [3]: [cr_returning_customer_sk#39, ca_state#46, sum#48] -Arguments: hashpartitioning(cr_returning_customer_sk#39, ca_state#46, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [3]: [cr_returning_customer_sk#40, ca_state#47, sum#49] +Arguments: hashpartitioning(cr_returning_customer_sk#40, ca_state#47, 5), ENSURE_REQUIREMENTS, [id=#50] (49) HashAggregate [codegen id : 18] -Input [3]: [cr_returning_customer_sk#39, ca_state#46, sum#48] -Keys [2]: [cr_returning_customer_sk#39, ca_state#46] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#41))#50] -Results [2]: [ca_state#46 AS ctr_state#36, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#41))#50,17,2) AS ctr_total_return#37] +Input [3]: [cr_returning_customer_sk#40, ca_state#47, sum#49] +Keys [2]: [cr_returning_customer_sk#40, ca_state#47] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#42))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#42))#51] +Results [2]: [ca_state#47 AS ctr_state#36, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#42))#51,17,2) AS ctr_total_return#37] (50) HashAggregate [codegen id : 18] Input [2]: [ctr_state#36, ctr_total_return#37] Keys [1]: [ctr_state#36] Functions [1]: [partial_avg(ctr_total_return#37)] -Aggregate Attributes [2]: [sum#51, count#52] -Results [3]: [ctr_state#36, sum#53, count#54] +Aggregate Attributes [2]: [sum#52, count#53] +Results [3]: [ctr_state#36, sum#54, count#55] (51) Exchange -Input [3]: [ctr_state#36, sum#53, count#54] -Arguments: hashpartitioning(ctr_state#36, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [3]: [ctr_state#36, sum#54, count#55] +Arguments: hashpartitioning(ctr_state#36, 5), ENSURE_REQUIREMENTS, [id=#56] (52) HashAggregate [codegen id : 19] -Input [3]: [ctr_state#36, sum#53, count#54] +Input [3]: [ctr_state#36, sum#54, count#55] Keys [1]: [ctr_state#36] Functions [1]: [avg(ctr_total_return#37)] -Aggregate Attributes [1]: [avg(ctr_total_return#37)#56] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#37)#56) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#57, ctr_state#36 AS ctr_state#36#58] +Aggregate Attributes [1]: [avg(ctr_total_return#37)#57] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#37)#57) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#58, ctr_state#36 AS ctr_state#36#59] (53) Filter [codegen id : 19] -Input [2]: [(avg(ctr_total_return) * 1.2)#57, ctr_state#36#58] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#57) +Input [2]: [(avg(ctr_total_return) * 1.2)#58, ctr_state#36#59] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#58) (54) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#57, ctr_state#36#58] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#59] +Input [2]: [(avg(ctr_total_return) * 1.2)#58, ctr_state#36#59] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#60] (55) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ctr_state#36] -Right keys [1]: [ctr_state#36#58] -Join condition: (cast(ctr_total_return#37 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#57) +Right keys [1]: [ctr_state#36#59] +Join condition: (CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#39 > (avg(ctr_total_return) * 1.2)#58) (56) Project [codegen id : 20] Output [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#37] -Input [19]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#36, ctr_total_return#37, (avg(ctr_total_return) * 1.2)#57, ctr_state#36#58] +Input [20]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#36, ctr_total_return#37, CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#39, (avg(ctr_total_return) * 1.2)#58, ctr_state#36#59] (57) TakeOrderedAndProject Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#37] @@ -331,27 +331,27 @@ BroadcastExchange (62) (58) Scan parquet default.date_dim -Output [2]: [d_date_sk#26, d_year#60] +Output [2]: [d_date_sk#26, d_year#61] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#26, d_year#60] +Input [2]: [d_date_sk#26, d_year#61] (60) Filter [codegen id : 1] -Input [2]: [d_date_sk#26, d_year#60] -Condition : ((isnotnull(d_year#60) AND (d_year#60 = 2000)) AND isnotnull(d_date_sk#26)) +Input [2]: [d_date_sk#26, d_year#61] +Condition : ((isnotnull(d_year#61) AND (d_year#61 = 2000)) AND isnotnull(d_date_sk#26)) (61) Project [codegen id : 1] Output [1]: [d_date_sk#26] -Input [2]: [d_date_sk#26, d_year#60] +Input [2]: [d_date_sk#26, d_year#61] (62) BroadcastExchange Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#62] -Subquery:2 Hosting operator id = 35 Hosting Expression = cr_returned_date_sk#42 IN dynamicpruning#25 +Subquery:2 Hosting operator id = 35 Hosting Expression = cr_returned_date_sk#43 IN dynamicpruning#25 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt index bca54597ee97f..bbe758caf12b2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] WholeStageCodegen (20) Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] + BroadcastHashJoin [ctr_state,ctr_state,CAST(ctr1.ctr_total_return AS DECIMAL(24,7)),(avg(ctr_total_return) * 1.2)] Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_state,ctr_total_return] SortMergeJoin [c_customer_sk,ctr_customer_sk] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt index 4a9a9cc153d0e..264674f7e825b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt @@ -1,52 +1,53 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.catalog_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.customer_address (7) - : : +- BroadcastExchange (33) - : : +- * Filter (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * HashAggregate (28) - : : +- Exchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * Filter (19) - : : : : +- * ColumnarToRow (18) - : : : : +- Scan parquet default.catalog_returns (17) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : +- BroadcastExchange (39) - : +- * Filter (38) - : +- * ColumnarToRow (37) - : +- Scan parquet default.customer (36) - +- BroadcastExchange (45) - +- * Filter (44) - +- * ColumnarToRow (43) - +- Scan parquet default.customer_address (42) +TakeOrderedAndProject (49) ++- * Project (48) + +- * BroadcastHashJoin Inner BuildRight (47) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Project (17) + : : : +- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.catalog_returns (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.customer_address (7) + : : +- BroadcastExchange (34) + : : +- * Filter (33) + : : +- * HashAggregate (32) + : : +- Exchange (31) + : : +- * HashAggregate (30) + : : +- * HashAggregate (29) + : : +- Exchange (28) + : : +- * HashAggregate (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (23) + : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : :- * Filter (20) + : : : : +- * ColumnarToRow (19) + : : : : +- Scan parquet default.catalog_returns (18) + : : : +- ReusedExchange (21) + : : +- ReusedExchange (24) + : +- BroadcastExchange (40) + : +- * Filter (39) + : +- * ColumnarToRow (38) + : +- Scan parquet default.customer (37) + +- BroadcastExchange (46) + +- * Filter (45) + +- * ColumnarToRow (44) + +- Scan parquet default.customer_address (43) (1) Scan parquet default.catalog_returns @@ -64,7 +65,7 @@ Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_in Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) -(4) ReusedExchange [Reuses operator id: 53] +(4) ReusedExchange [Reuses operator id: 54] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -125,188 +126,192 @@ Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#14, ca_state#8 AS ct Input [3]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16] Condition : isnotnull(ctr_total_return#16) -(17) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, cr_returned_date_sk#20] +(17) Project [codegen id : 11] +Output [4]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, cast(ctr_total_return#16 as decimal(24,7)) AS CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#17] +Input [3]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16] + +(18) Scan parquet default.catalog_returns +Output [4]: [cr_returning_customer_sk#18, cr_returning_addr_sk#19, cr_return_amt_inc_tax#20, cr_returned_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#20), dynamicpruningexpression(cr_returned_date_sk#20 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#21), dynamicpruningexpression(cr_returned_date_sk#21 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, cr_returned_date_sk#20] +(19) ColumnarToRow [codegen id : 6] +Input [4]: [cr_returning_customer_sk#18, cr_returning_addr_sk#19, cr_return_amt_inc_tax#20, cr_returned_date_sk#21] -(19) Filter [codegen id : 6] -Input [4]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, cr_returned_date_sk#20] -Condition : isnotnull(cr_returning_addr_sk#18) +(20) Filter [codegen id : 6] +Input [4]: [cr_returning_customer_sk#18, cr_returning_addr_sk#19, cr_return_amt_inc_tax#20, cr_returned_date_sk#21] +Condition : isnotnull(cr_returning_addr_sk#19) -(20) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#21] +(21) ReusedExchange [Reuses operator id: 54] +Output [1]: [d_date_sk#22] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#20] -Right keys [1]: [d_date_sk#21] +(22) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cr_returned_date_sk#21] +Right keys [1]: [d_date_sk#22] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19] -Input [5]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, cr_returned_date_sk#20, d_date_sk#21] +(23) Project [codegen id : 6] +Output [3]: [cr_returning_customer_sk#18, cr_returning_addr_sk#19, cr_return_amt_inc_tax#20] +Input [5]: [cr_returning_customer_sk#18, cr_returning_addr_sk#19, cr_return_amt_inc_tax#20, cr_returned_date_sk#21, d_date_sk#22] -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#22, ca_state#23] +(24) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#23, ca_state#24] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#18] -Right keys [1]: [ca_address_sk#22] +(25) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cr_returning_addr_sk#19] +Right keys [1]: [ca_address_sk#23] Join condition: None -(25) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#17, cr_return_amt_inc_tax#19, ca_state#23] -Input [5]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, ca_address_sk#22, ca_state#23] +(26) Project [codegen id : 6] +Output [3]: [cr_returning_customer_sk#18, cr_return_amt_inc_tax#20, ca_state#24] +Input [5]: [cr_returning_customer_sk#18, cr_returning_addr_sk#19, cr_return_amt_inc_tax#20, ca_address_sk#23, ca_state#24] -(26) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#17, cr_return_amt_inc_tax#19, ca_state#23] -Keys [2]: [cr_returning_customer_sk#17, ca_state#23] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#19))] -Aggregate Attributes [1]: [sum#24] -Results [3]: [cr_returning_customer_sk#17, ca_state#23, sum#25] +(27) HashAggregate [codegen id : 6] +Input [3]: [cr_returning_customer_sk#18, cr_return_amt_inc_tax#20, ca_state#24] +Keys [2]: [cr_returning_customer_sk#18, ca_state#24] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#20))] +Aggregate Attributes [1]: [sum#25] +Results [3]: [cr_returning_customer_sk#18, ca_state#24, sum#26] -(27) Exchange -Input [3]: [cr_returning_customer_sk#17, ca_state#23, sum#25] -Arguments: hashpartitioning(cr_returning_customer_sk#17, ca_state#23, 5), ENSURE_REQUIREMENTS, [id=#26] - -(28) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#17, ca_state#23, sum#25] -Keys [2]: [cr_returning_customer_sk#17, ca_state#23] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#19))#27] -Results [2]: [ca_state#23 AS ctr_state#15, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#19))#27,17,2) AS ctr_total_return#16] +(28) Exchange +Input [3]: [cr_returning_customer_sk#18, ca_state#24, sum#26] +Arguments: hashpartitioning(cr_returning_customer_sk#18, ca_state#24, 5), ENSURE_REQUIREMENTS, [id=#27] (29) HashAggregate [codegen id : 7] +Input [3]: [cr_returning_customer_sk#18, ca_state#24, sum#26] +Keys [2]: [cr_returning_customer_sk#18, ca_state#24] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#20))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#20))#28] +Results [2]: [ca_state#24 AS ctr_state#15, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#20))#28,17,2) AS ctr_total_return#16] + +(30) HashAggregate [codegen id : 7] Input [2]: [ctr_state#15, ctr_total_return#16] Keys [1]: [ctr_state#15] Functions [1]: [partial_avg(ctr_total_return#16)] -Aggregate Attributes [2]: [sum#28, count#29] -Results [3]: [ctr_state#15, sum#30, count#31] +Aggregate Attributes [2]: [sum#29, count#30] +Results [3]: [ctr_state#15, sum#31, count#32] -(30) Exchange -Input [3]: [ctr_state#15, sum#30, count#31] -Arguments: hashpartitioning(ctr_state#15, 5), ENSURE_REQUIREMENTS, [id=#32] +(31) Exchange +Input [3]: [ctr_state#15, sum#31, count#32] +Arguments: hashpartitioning(ctr_state#15, 5), ENSURE_REQUIREMENTS, [id=#33] -(31) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#15, sum#30, count#31] +(32) HashAggregate [codegen id : 8] +Input [3]: [ctr_state#15, sum#31, count#32] Keys [1]: [ctr_state#15] Functions [1]: [avg(ctr_total_return#16)] -Aggregate Attributes [1]: [avg(ctr_total_return#16)#33] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#16)#33) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#34, ctr_state#15 AS ctr_state#15#35] +Aggregate Attributes [1]: [avg(ctr_total_return#16)#34] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#16)#34) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#35, ctr_state#15 AS ctr_state#15#36] -(32) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#34) +(33) Filter [codegen id : 8] +Input [2]: [(avg(ctr_total_return) * 1.2)#35, ctr_state#15#36] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#35) -(33) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#36] +(34) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#35, ctr_state#15#36] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#37] -(34) BroadcastHashJoin [codegen id : 11] +(35) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#15] -Right keys [1]: [ctr_state#15#35] -Join condition: (cast(ctr_total_return#16 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#34) +Right keys [1]: [ctr_state#15#36] +Join condition: (CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#17 > (avg(ctr_total_return) * 1.2)#35) -(35) Project [codegen id : 11] +(36) Project [codegen id : 11] Output [2]: [ctr_customer_sk#14, ctr_total_return#16] -Input [5]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, (avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] +Input [6]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#17, (avg(ctr_total_return) * 1.2)#35, ctr_state#15#36] -(36) Scan parquet default.customer -Output [6]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] +(37) Scan parquet default.customer +Output [6]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] +(38) ColumnarToRow [codegen id : 9] +Input [6]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43] -(38) Filter [codegen id : 9] -Input [6]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] -Condition : (isnotnull(c_customer_sk#37) AND isnotnull(c_current_addr_sk#39)) +(39) Filter [codegen id : 9] +Input [6]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43] +Condition : (isnotnull(c_customer_sk#38) AND isnotnull(c_current_addr_sk#40)) -(39) BroadcastExchange -Input [6]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] +(40) BroadcastExchange +Input [6]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] -(40) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#14] -Right keys [1]: [c_customer_sk#37] +Right keys [1]: [c_customer_sk#38] Join condition: None -(41) Project [codegen id : 11] -Output [6]: [ctr_total_return#16, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] -Input [8]: [ctr_customer_sk#14, ctr_total_return#16, c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] +(42) Project [codegen id : 11] +Output [6]: [ctr_total_return#16, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43] +Input [8]: [ctr_customer_sk#14, ctr_total_return#16, c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43] -(42) Scan parquet default.customer_address -Output [12]: [ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] +(43) Scan parquet default.customer_address +Output [12]: [ca_address_sk#45, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] +(44) ColumnarToRow [codegen id : 10] +Input [12]: [ca_address_sk#45, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56] -(44) Filter [codegen id : 10] -Input [12]: [ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] -Condition : ((isnotnull(ca_state#51) AND (ca_state#51 = GA)) AND isnotnull(ca_address_sk#44)) +(45) Filter [codegen id : 10] +Input [12]: [ca_address_sk#45, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56] +Condition : ((isnotnull(ca_state#52) AND (ca_state#52 = GA)) AND isnotnull(ca_address_sk#45)) -(45) BroadcastExchange -Input [12]: [ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#56] +(46) BroadcastExchange +Input [12]: [ca_address_sk#45, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#57] -(46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#39] -Right keys [1]: [ca_address_sk#44] +(47) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#40] +Right keys [1]: [ca_address_sk#45] Join condition: None -(47) Project [codegen id : 11] -Output [16]: [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55, ctr_total_return#16] -Input [18]: [ctr_total_return#16, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] +(48) Project [codegen id : 11] +Output [16]: [c_customer_id#39, c_salutation#41, c_first_name#42, c_last_name#43, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56, ctr_total_return#16] +Input [18]: [ctr_total_return#16, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, ca_address_sk#45, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56] -(48) TakeOrderedAndProject -Input [16]: [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55, ctr_total_return#16] -Arguments: 100, [c_customer_id#38 ASC NULLS FIRST, c_salutation#40 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, c_last_name#42 ASC NULLS FIRST, ca_street_number#45 ASC NULLS FIRST, ca_street_name#46 ASC NULLS FIRST, ca_street_type#47 ASC NULLS FIRST, ca_suite_number#48 ASC NULLS FIRST, ca_city#49 ASC NULLS FIRST, ca_county#50 ASC NULLS FIRST, ca_state#51 ASC NULLS FIRST, ca_zip#52 ASC NULLS FIRST, ca_country#53 ASC NULLS FIRST, ca_gmt_offset#54 ASC NULLS FIRST, ca_location_type#55 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55, ctr_total_return#16] +(49) TakeOrderedAndProject +Input [16]: [c_customer_id#39, c_salutation#41, c_first_name#42, c_last_name#43, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56, ctr_total_return#16] +Arguments: 100, [c_customer_id#39 ASC NULLS FIRST, c_salutation#41 ASC NULLS FIRST, c_first_name#42 ASC NULLS FIRST, c_last_name#43 ASC NULLS FIRST, ca_street_number#46 ASC NULLS FIRST, ca_street_name#47 ASC NULLS FIRST, ca_street_type#48 ASC NULLS FIRST, ca_suite_number#49 ASC NULLS FIRST, ca_city#50 ASC NULLS FIRST, ca_county#51 ASC NULLS FIRST, ca_state#52 ASC NULLS FIRST, ca_zip#53 ASC NULLS FIRST, ca_country#54 ASC NULLS FIRST, ca_gmt_offset#55 ASC NULLS FIRST, ca_location_type#56 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#39, c_salutation#41, c_first_name#42, c_last_name#43, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56, ctr_total_return#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (53) -+- * Project (52) - +- * Filter (51) - +- * ColumnarToRow (50) - +- Scan parquet default.date_dim (49) +BroadcastExchange (54) ++- * Project (53) + +- * Filter (52) + +- * ColumnarToRow (51) + +- Scan parquet default.date_dim (50) -(49) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#57] +(50) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_year#58] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(50) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#57] +(51) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#58] -(51) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#57] -Condition : ((isnotnull(d_year#57) AND (d_year#57 = 2000)) AND isnotnull(d_date_sk#6)) +(52) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#58] +Condition : ((isnotnull(d_year#58) AND (d_year#58 = 2000)) AND isnotnull(d_date_sk#6)) -(52) Project [codegen id : 1] +(53) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#57] +Input [2]: [d_date_sk#6, d_year#58] -(53) BroadcastExchange +(54) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#59] -Subquery:2 Hosting operator id = 17 Hosting Expression = cr_returned_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 Hosting Expression = cr_returned_date_sk#21 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt index af8c23b5f7a00..354d405ee12c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -5,38 +5,39 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returning_addr_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [ca_address_sk,ca_state] + BroadcastHashJoin [ctr_state,ctr_state,CAST(ctr1.ctr_total_return AS DECIMAL(24,7)),(avg(ctr_total_return) * 1.2)] + Project [ctr_customer_sk,ctr_state,ctr_total_return] + Filter [ctr_total_return] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] + InputAdapter + Exchange [cr_returning_customer_sk,ca_state] #1 + WholeStageCodegen (3) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Filter [cr_returning_addr_sk,cr_returning_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] + Scan parquet default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Filter [ca_address_sk,ca_state] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt index d13b0f1c9bb91..3b653ecf5d846 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == -* HashAggregate (29) -+- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildLeft (22) +* HashAggregate (30) ++- Exchange (29) + +- * HashAggregate (28) + +- * Project (27) + +- * BroadcastHashJoin Inner BuildRight (26) + :- * Project (24) + : +- * BroadcastHashJoin Inner BuildLeft (23) : :- BroadcastExchange (18) : : +- * Project (17) : : +- * BroadcastHashJoin Inner BuildLeft (16) @@ -24,10 +24,11 @@ : : : +- * ColumnarToRow (7) : : : +- Scan parquet default.web_sales (6) : : +- ReusedExchange (9) - : +- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet default.web_sales (19) - +- ReusedExchange (24) + : +- * Project (22) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.web_sales (19) + +- ReusedExchange (25) (1) Scan parquet default.item @@ -67,7 +68,7 @@ Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] Condition : isnotnull(ws_item_sk#4) -(9) ReusedExchange [Reuses operator id: 34] +(9) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#8] (10) BroadcastHashJoin [codegen id : 3] @@ -129,76 +130,80 @@ Input [3]: [ws_item_sk#17, ws_ext_discount_amt#18, ws_sold_date_sk#19] Input [3]: [ws_item_sk#17, ws_ext_discount_amt#18, ws_sold_date_sk#19] Condition : (isnotnull(ws_item_sk#17) AND isnotnull(ws_ext_discount_amt#18)) -(22) BroadcastHashJoin [codegen id : 6] +(22) Project +Output [4]: [ws_item_sk#17, ws_ext_discount_amt#18, ws_sold_date_sk#19, cast(ws_ext_discount_amt#18 as decimal(14,7)) AS CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7))#20] +Input [3]: [ws_item_sk#17, ws_ext_discount_amt#18, ws_sold_date_sk#19] + +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] Right keys [1]: [ws_item_sk#17] -Join condition: (cast(ws_ext_discount_amt#18 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#15) +Join condition: (CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7))#20 > (1.3 * avg(ws_ext_discount_amt))#15) -(23) Project [codegen id : 6] +(24) Project [codegen id : 6] Output [2]: [ws_ext_discount_amt#18, ws_sold_date_sk#19] -Input [5]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#17, ws_ext_discount_amt#18, ws_sold_date_sk#19] +Input [6]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#17, ws_ext_discount_amt#18, ws_sold_date_sk#19, CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7))#20] -(24) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#20] +(25) ReusedExchange [Reuses operator id: 35] +Output [1]: [d_date_sk#21] -(25) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None -(26) Project [codegen id : 6] +(27) Project [codegen id : 6] Output [1]: [ws_ext_discount_amt#18] -Input [3]: [ws_ext_discount_amt#18, ws_sold_date_sk#19, d_date_sk#20] +Input [3]: [ws_ext_discount_amt#18, ws_sold_date_sk#19, d_date_sk#21] -(27) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 6] Input [1]: [ws_ext_discount_amt#18] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#18))] -Aggregate Attributes [1]: [sum#21] -Results [1]: [sum#22] +Aggregate Attributes [1]: [sum#22] +Results [1]: [sum#23] -(28) Exchange -Input [1]: [sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] +(29) Exchange +Input [1]: [sum#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] -(29) HashAggregate [codegen id : 7] -Input [1]: [sum#22] +(30) HashAggregate [codegen id : 7] +Input [1]: [sum#23] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#18))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#18))#24,17,2) AS Excess Discount Amount #25] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#18))#25] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#18))#25,17,2) AS Excess Discount Amount #26] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (34) -+- * Project (33) - +- * Filter (32) - +- * ColumnarToRow (31) - +- Scan parquet default.date_dim (30) +BroadcastExchange (35) ++- * Project (34) + +- * Filter (33) + +- * ColumnarToRow (32) + +- Scan parquet default.date_dim (31) -(30) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#26] +(31) Scan parquet default.date_dim +Output [2]: [d_date_sk#8, d_date#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#8, d_date#26] +(32) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#8, d_date#27] -(32) Filter [codegen id : 1] -Input [2]: [d_date_sk#8, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 2000-01-27)) AND (d_date#26 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) +(33) Filter [codegen id : 1] +Input [2]: [d_date_sk#8, d_date#27] +Condition : (((isnotnull(d_date#27) AND (d_date#27 >= 2000-01-27)) AND (d_date#27 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) -(33) Project [codegen id : 1] +(34) Project [codegen id : 1] Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#26] +Input [2]: [d_date_sk#8, d_date#27] -(34) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt index abf6a164982a1..03b8cf105e863 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt @@ -7,7 +7,7 @@ WholeStageCodegen (7) Project [ws_ext_discount_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_ext_discount_amt,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] + BroadcastHashJoin [i_item_sk,ws_item_sk,CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7)),(1.3 * avg(ws_ext_discount_amt))] InputAdapter BroadcastExchange #2 WholeStageCodegen (4) @@ -43,10 +43,11 @@ WholeStageCodegen (7) Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #5 - Filter [ws_item_sk,ws_ext_discount_amt] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Project [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + Filter [ws_item_sk,ws_ext_discount_amt] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index 72c206a372644..9246fd0f322cb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -73,100 +73,100 @@ Right keys [1]: [i_item_sk#5] Join condition: None (10) Project [codegen id : 6] -Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] +Output [4]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, cast(ws_ext_discount_amt#2 as decimal(14,7)) AS CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7))#8] Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.web_sales -Output [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] +Output [3]: [ws_item_sk#9, ws_ext_discount_amt#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] +Input [3]: [ws_item_sk#9, ws_ext_discount_amt#10, ws_sold_date_sk#11] (13) Filter [codegen id : 3] -Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] -Condition : isnotnull(ws_item_sk#8) +Input [3]: [ws_item_sk#9, ws_ext_discount_amt#10, ws_sold_date_sk#11] +Condition : isnotnull(ws_item_sk#9) (14) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#12] (15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#10] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ws_sold_date_sk#11] +Right keys [1]: [d_date_sk#12] Join condition: None (16) Project [codegen id : 3] -Output [2]: [ws_item_sk#8, ws_ext_discount_amt#9] -Input [4]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10, d_date_sk#11] +Output [2]: [ws_item_sk#9, ws_ext_discount_amt#10] +Input [4]: [ws_item_sk#9, ws_ext_discount_amt#10, ws_sold_date_sk#11, d_date_sk#12] (17) HashAggregate [codegen id : 3] -Input [2]: [ws_item_sk#8, ws_ext_discount_amt#9] -Keys [1]: [ws_item_sk#8] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#9))] -Aggregate Attributes [2]: [sum#12, count#13] -Results [3]: [ws_item_sk#8, sum#14, count#15] +Input [2]: [ws_item_sk#9, ws_ext_discount_amt#10] +Keys [1]: [ws_item_sk#9] +Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#10))] +Aggregate Attributes [2]: [sum#13, count#14] +Results [3]: [ws_item_sk#9, sum#15, count#16] (18) Exchange -Input [3]: [ws_item_sk#8, sum#14, count#15] -Arguments: hashpartitioning(ws_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [3]: [ws_item_sk#9, sum#15, count#16] +Arguments: hashpartitioning(ws_item_sk#9, 5), ENSURE_REQUIREMENTS, [id=#17] (19) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#8, sum#14, count#15] -Keys [1]: [ws_item_sk#8] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))#17] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#9))#17 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#18, ws_item_sk#8] +Input [3]: [ws_item_sk#9, sum#15, count#16] +Keys [1]: [ws_item_sk#9] +Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#10))] +Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#10))#18] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#10))#18 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#19, ws_item_sk#9] (20) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#18, ws_item_sk#8] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#18) +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#19, ws_item_sk#9] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#19) (21) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#18, ws_item_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#19, ws_item_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#20] (22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [ws_item_sk#8] -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#18) +Right keys [1]: [ws_item_sk#9] +Join condition: (CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7))#8 > (1.3 * avg(ws_ext_discount_amt))#19) (23) Project [codegen id : 6] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#18, ws_item_sk#8] +Input [6]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7))#8, (1.3 * avg(ws_ext_discount_amt))#19, ws_item_sk#9] (24) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#21] (25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None (26) Project [codegen id : 6] Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#20] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#21] (27) HashAggregate [codegen id : 6] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#21] -Results [1]: [sum#22] +Aggregate Attributes [1]: [sum#22] +Results [1]: [sum#23] (28) Exchange -Input [1]: [sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] +Input [1]: [sum#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] (29) HashAggregate [codegen id : 7] -Input [1]: [sum#22] +Input [1]: [sum#23] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#24,17,2) AS Excess Discount Amount #25] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#25] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#25,17,2) AS Excess Discount Amount #26] ===== Subqueries ===== @@ -179,27 +179,27 @@ BroadcastExchange (34) (30) Scan parquet default.date_dim -Output [2]: [d_date_sk#20, d_date#26] +Output [2]: [d_date_sk#21, d_date#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_date#26] +Input [2]: [d_date_sk#21, d_date#27] (32) Filter [codegen id : 1] -Input [2]: [d_date_sk#20, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 2000-01-27)) AND (d_date#26 <= 2000-04-26)) AND isnotnull(d_date_sk#20)) +Input [2]: [d_date_sk#21, d_date#27] +Condition : (((isnotnull(d_date#27) AND (d_date#27 >= 2000-01-27)) AND (d_date#27 <= 2000-04-26)) AND isnotnull(d_date_sk#21)) (33) Project [codegen id : 1] -Output [1]: [d_date_sk#20] -Input [2]: [d_date_sk#20, d_date#26] +Output [1]: [d_date_sk#21] +Input [2]: [d_date_sk#21, d_date#27] (34) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] +Input [1]: [d_date_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index 86ade79511d56..291f18fe08f60 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -7,7 +7,7 @@ WholeStageCodegen (7) Project [ws_ext_discount_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_ext_discount_amt,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] + BroadcastHashJoin [i_item_sk,ws_item_sk,CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7)),(1.3 * avg(ws_ext_discount_amt))] Project [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_ext_discount_amt] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt index 8e15a8b4c1e2a..84342b5469473 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt @@ -350,94 +350,94 @@ Right keys [1]: [customer_id#72] Join condition: None (60) Project [codegen id : 26] -Output [8]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, year_total#73] +Output [7]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#73, CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#75] Input [9]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, customer_id#72, year_total#73] (61) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] +Output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_sold_date_sk#78 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] +Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] (63) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] -Condition : isnotnull(ws_bill_customer_sk#75) +Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_bill_customer_sk#76) (64) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#79, d_year#80] +Output [2]: [d_date_sk#80, d_year#81] (65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#78] -Right keys [1]: [d_date_sk#79] +Left keys [1]: [ws_sold_date_sk#79] +Right keys [1]: [d_date_sk#80] Join condition: None (66) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Input [6]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78, d_date_sk#79, d_year#80] +Output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#80, d_year#81] (67) Exchange -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Arguments: hashpartitioning(ws_bill_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] +Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] (68) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Arguments: [ws_bill_customer_sk#75 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] +Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 (69) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] +Output [8]: [c_customer_sk#83, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90] (70) Sort [codegen id : 31] -Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#83, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90] +Arguments: [c_customer_sk#83 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#75] -Right keys [1]: [c_customer_sk#82] +Left keys [1]: [ws_bill_customer_sk#76] +Right keys [1]: [c_customer_sk#83] Join condition: None (72) Project [codegen id : 32] -Output [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Input [12]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80, c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] +Output [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] +Input [12]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81, c_customer_sk#83, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90] (73) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#90] -Results [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Input [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] +Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#91] +Results [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, sum#92] (74) Exchange -Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] -Arguments: hashpartitioning(c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, sum#92] +Arguments: hashpartitioning(c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, 5), ENSURE_REQUIREMENTS, [id=#93] (75) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] -Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93] -Results [2]: [c_customer_id#83 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] +Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, sum#92] +Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))#94] +Results [2]: [c_customer_id#84 AS customer_id#95, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))#94,18,2) AS year_total#96] (76) Exchange -Input [2]: [customer_id#94, year_total#95] -Arguments: hashpartitioning(customer_id#94, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [2]: [customer_id#95, year_total#96] +Arguments: hashpartitioning(customer_id#95, 5), ENSURE_REQUIREMENTS, [id=#97] (77) Sort [codegen id : 34] -Input [2]: [customer_id#94, year_total#95] -Arguments: [customer_id#94 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#95, year_total#96] +Arguments: [customer_id#95 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#94] -Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END) +Right keys [1]: [customer_id#95] +Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#96) / promote_precision(year_total#73)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#75) (79) Project [codegen id : 35] Output [4]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48] -Input [10]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, year_total#73, customer_id#94, year_total#95] +Input [9]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#73, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#75, customer_id#95, year_total#96] (80) TakeOrderedAndProject Input [4]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48] @@ -468,7 +468,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2001)) AND isnotnull(d_date_sk (84) BroadcastExchange Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#98] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (88) @@ -493,10 +493,10 @@ Condition : ((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND isnotnull(d_date_ (88) BroadcastExchange Input [2]: [d_date_sk#30, d_year#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#78 IN dynamicpruning#29 +Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#29 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt index b3f8a57ba0f5b..35e8fb614399f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] WholeStageCodegen (35) Project [customer_id,customer_first_name,customer_last_name,customer_email_address] - SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END] InputAdapter WholeStageCodegen (26) - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] + Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt index da1fd1cd155aa..bd1baae66cff9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt @@ -320,93 +320,93 @@ Right keys [1]: [customer_id#70] Join condition: None (53) Project [codegen id : 16] -Output [8]: [customer_id#21, year_total#22, customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#47, year_total#71] +Output [7]: [customer_id#21, customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#71, CASE WHEN (year_total#22 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#22)), DecimalType(38,20), true) ELSE 0E-20 END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#73] Input [9]: [customer_id#21, year_total#22, customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#47, customer_id#70, year_total#71] (54) Scan parquet default.customer -Output [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Output [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (55) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] (56) Filter [codegen id : 14] -Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] -Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_customer_id#74)) +Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] +Condition : (isnotnull(c_customer_sk#74) AND isnotnull(c_customer_id#75)) (57) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Output [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#84), dynamicpruningexpression(ws_sold_date_sk#84 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_sold_date_sk#85 IN dynamicpruning#35)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] (59) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] -Condition : isnotnull(ws_bill_customer_sk#81) +Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] +Condition : isnotnull(ws_bill_customer_sk#82) (60) BroadcastExchange -Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#85] +Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#86] (61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#73] -Right keys [1]: [ws_bill_customer_sk#81] +Left keys [1]: [c_customer_sk#74] +Right keys [1]: [ws_bill_customer_sk#82] Join condition: None (62) Project [codegen id : 14] -Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] -Input [12]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Output [10]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] +Input [12]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] (63) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#86, d_year#87] +Output [2]: [d_date_sk#87, d_year#88] (64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#84] -Right keys [1]: [d_date_sk#86] +Left keys [1]: [ws_sold_date_sk#85] +Right keys [1]: [d_date_sk#87] Join condition: None (65) Project [codegen id : 14] -Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] -Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84, d_date_sk#86, d_year#87] +Output [10]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, ws_ext_discount_amt#83, ws_ext_list_price#84, d_year#88] +Input [12]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85, d_date_sk#87, d_year#88] (66) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] -Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#88] -Results [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] +Input [10]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, ws_ext_discount_amt#83, ws_ext_list_price#84, d_year#88] +Keys [8]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, d_year#88] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#84 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#83 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#89] +Results [9]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, d_year#88, sum#90] (67) Exchange -Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] -Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [9]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, d_year#88, sum#90] +Arguments: hashpartitioning(c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, d_year#88, 5), ENSURE_REQUIREMENTS, [id=#91] (68) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] -Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91] -Results [2]: [c_customer_id#74 AS customer_id#92, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91,18,2) AS year_total#93] +Input [9]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, d_year#88, sum#90] +Keys [8]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, d_year#88] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#84 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#83 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#84 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#83 as decimal(8,2)))), DecimalType(8,2), true)))#92] +Results [2]: [c_customer_id#75 AS customer_id#93, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#84 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#83 as decimal(8,2)))), DecimalType(8,2), true)))#92,18,2) AS year_total#94] (69) BroadcastExchange -Input [2]: [customer_id#92, year_total#93] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#94] +Input [2]: [customer_id#93, year_total#94] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#95] (70) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#21] -Right keys [1]: [customer_id#92] -Join condition: (CASE WHEN (year_total#71 > 0.00) THEN CheckOverflow((promote_precision(year_total#93) / promote_precision(year_total#71)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#22 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#22)), DecimalType(38,20), true) ELSE 0E-20 END) +Right keys [1]: [customer_id#93] +Join condition: (CASE WHEN (year_total#71 > 0.00) THEN CheckOverflow((promote_precision(year_total#94) / promote_precision(year_total#71)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#73) (71) Project [codegen id : 16] Output [4]: [customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46] -Input [10]: [customer_id#21, year_total#22, customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#47, year_total#71, customer_id#92, year_total#93] +Input [9]: [customer_id#21, customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#71, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#73, customer_id#93, year_total#94] (72) TakeOrderedAndProject Input [4]: [customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46] @@ -437,7 +437,7 @@ Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2001)) AND isnotnull(d_date_ (76) BroadcastExchange Input [2]: [d_date_sk#15, d_year#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#34 IN dynamicpruning#35 BroadcastExchange (80) @@ -462,10 +462,10 @@ Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_ (80) BroadcastExchange Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#97] Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#35 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#85 IN dynamicpruning#35 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index 6ebf857eba13a..b313db49b564e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -1,8 +1,8 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] WholeStageCodegen (16) Project [customer_id,customer_first_name,customer_last_name,customer_email_address] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END] + Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt index c8cd870c62c8a..bf54f30085860 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt @@ -91,345 +91,411 @@ Right keys [1]: [ca_zip#9] Join condition: None (10) Project [codegen id : 2] -Output [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] +Output [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, upper(ca_country#10) AS upper(spark_catalog.default.customer_address.ca_country)#11] Input [8]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5, ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] (11) BroadcastExchange -Input [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] -Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#11] +Input [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, upper(spark_catalog.default.customer_address.ca_country)#11] +Arguments: HashedRelationBroadcastMode(List(input[3, int, true], input[5, string, true]),false), [id=#12] (12) Scan parquet default.customer -Output [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Output [5]: [c_customer_sk#13, c_current_addr_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (13) ColumnarToRow -Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Input [5]: [c_customer_sk#13, c_current_addr_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (14) Filter -Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] -Condition : ((isnotnull(c_customer_sk#12) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_birth_country#16)) +Input [5]: [c_customer_sk#13, c_current_addr_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Condition : ((isnotnull(c_customer_sk#13) AND isnotnull(c_current_addr_sk#14)) AND isnotnull(c_birth_country#17)) (15) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ca_address_sk#7, upper(ca_country#10)] -Right keys [2]: [c_current_addr_sk#13, c_birth_country#16] +Left keys [2]: [ca_address_sk#7, upper(spark_catalog.default.customer_address.ca_country)#11] +Right keys [2]: [c_current_addr_sk#14, c_birth_country#17] Join condition: None (16) Project [codegen id : 3] -Output [7]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#12, c_first_name#14, c_last_name#15] -Input [11]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10, c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Output [7]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#13, c_first_name#15, c_last_name#16] +Input [11]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, upper(spark_catalog.default.customer_address.ca_country)#11, c_customer_sk#13, c_current_addr_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (17) BroadcastExchange -Input [7]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#12, c_first_name#14, c_last_name#15] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#17] +Input [7]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#13, c_first_name#15, c_last_name#16] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#18] (18) Scan parquet default.store_sales -Output [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] +Output [6]: [ss_item_sk#19, ss_customer_sk#20, ss_store_sk#21, ss_ticket_number#22, ss_net_paid#23, ss_sold_date_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (19) ColumnarToRow -Input [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] +Input [6]: [ss_item_sk#19, ss_customer_sk#20, ss_store_sk#21, ss_ticket_number#22, ss_net_paid#23, ss_sold_date_sk#24] (20) Filter -Input [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : (((isnotnull(ss_ticket_number#21) AND isnotnull(ss_item_sk#18)) AND isnotnull(ss_store_sk#20)) AND isnotnull(ss_customer_sk#19)) +Input [6]: [ss_item_sk#19, ss_customer_sk#20, ss_store_sk#21, ss_ticket_number#22, ss_net_paid#23, ss_sold_date_sk#24] +Condition : (((isnotnull(ss_ticket_number#22) AND isnotnull(ss_item_sk#19)) AND isnotnull(ss_store_sk#21)) AND isnotnull(ss_customer_sk#20)) (21) Project -Output [5]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22] -Input [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] +Output [5]: [ss_item_sk#19, ss_customer_sk#20, ss_store_sk#21, ss_ticket_number#22, ss_net_paid#23] +Input [6]: [ss_item_sk#19, ss_customer_sk#20, ss_store_sk#21, ss_ticket_number#22, ss_net_paid#23, ss_sold_date_sk#24] (22) BroadcastHashJoin [codegen id : 5] -Left keys [2]: [s_store_sk#1, c_customer_sk#12] -Right keys [2]: [ss_store_sk#20, ss_customer_sk#19] +Left keys [2]: [s_store_sk#1, c_customer_sk#13] +Right keys [2]: [ss_store_sk#21, ss_customer_sk#20] Join condition: None (23) Project [codegen id : 5] -Output [8]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22] -Input [12]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#12, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22] +Output [8]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_ticket_number#22, ss_net_paid#23] +Input [12]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#13, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_customer_sk#20, ss_store_sk#21, ss_ticket_number#22, ss_net_paid#23] (24) Scan parquet default.item -Output [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Output [6]: [i_item_sk#25, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Input [6]: [i_item_sk#25, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] (26) Filter [codegen id : 4] -Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Condition : ((isnotnull(i_color#27) AND (i_color#27 = pale )) AND isnotnull(i_item_sk#24)) +Input [6]: [i_item_sk#25, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] +Condition : ((isnotnull(i_color#28) AND (i_color#28 = pale )) AND isnotnull(i_item_sk#25)) (27) BroadcastExchange -Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [6]: [i_item_sk#25, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#18] -Right keys [1]: [i_item_sk#24] +Left keys [1]: [ss_item_sk#19] +Right keys [1]: [i_item_sk#25] Join condition: None (29) Project [codegen id : 5] -Output [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Input [14]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Output [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_ticket_number#22, ss_net_paid#23, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] +Input [14]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_ticket_number#22, ss_net_paid#23, i_item_sk#25, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] (30) Exchange -Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Arguments: hashpartitioning(ss_ticket_number#21, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_ticket_number#22, ss_net_paid#23, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] +Arguments: hashpartitioning(ss_ticket_number#22, ss_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#32] (31) Sort [codegen id : 6] -Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Arguments: [ss_ticket_number#21 ASC NULLS FIRST, ss_item_sk#18 ASC NULLS FIRST], false, 0 +Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_ticket_number#22, ss_net_paid#23, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] +Arguments: [ss_ticket_number#22 ASC NULLS FIRST, ss_item_sk#19 ASC NULLS FIRST], false, 0 (32) Scan parquet default.store_returns -Output [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Output [3]: [sr_item_sk#33, sr_ticket_number#34, sr_returned_date_sk#35] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (33) ColumnarToRow [codegen id : 7] -Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Input [3]: [sr_item_sk#33, sr_ticket_number#34, sr_returned_date_sk#35] (34) Filter [codegen id : 7] -Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [3]: [sr_item_sk#33, sr_ticket_number#34, sr_returned_date_sk#35] +Condition : (isnotnull(sr_ticket_number#34) AND isnotnull(sr_item_sk#33)) (35) Project [codegen id : 7] -Output [2]: [sr_item_sk#32, sr_ticket_number#33] -Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Output [2]: [sr_item_sk#33, sr_ticket_number#34] +Input [3]: [sr_item_sk#33, sr_ticket_number#34, sr_returned_date_sk#35] (36) Exchange -Input [2]: [sr_item_sk#32, sr_ticket_number#33] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [2]: [sr_item_sk#33, sr_ticket_number#34] +Arguments: hashpartitioning(sr_ticket_number#34, sr_item_sk#33, 5), ENSURE_REQUIREMENTS, [id=#36] (37) Sort [codegen id : 8] -Input [2]: [sr_item_sk#32, sr_ticket_number#33] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#33, sr_ticket_number#34] +Arguments: [sr_ticket_number#34 ASC NULLS FIRST, sr_item_sk#33 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#21, ss_item_sk#18] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [ss_ticket_number#22, ss_item_sk#19] +Right keys [2]: [sr_ticket_number#34, sr_item_sk#33] Join condition: None (39) Project [codegen id : 9] -Output [11]: [ss_net_paid#22, s_store_name#2, s_state#4, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, c_first_name#14, c_last_name#15, ca_state#8] -Input [15]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, sr_item_sk#32, sr_ticket_number#33] +Output [11]: [ss_net_paid#23, s_store_name#2, s_state#4, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30, c_first_name#15, c_last_name#16, ca_state#8] +Input [15]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_ticket_number#22, ss_net_paid#23, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30, sr_item_sk#33, sr_ticket_number#34] (40) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#22, s_store_name#2, s_state#4, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, c_first_name#14, c_last_name#15, ca_state#8] -Keys [10]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] -Aggregate Attributes [1]: [sum#36] -Results [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#37] +Input [11]: [ss_net_paid#23, s_store_name#2, s_state#4, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30, c_first_name#15, c_last_name#16, ca_state#8] +Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#2, ca_state#8, s_state#4, i_color#28, i_current_price#26, i_manager_id#30, i_units#29, i_size#27] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#23))] +Aggregate Attributes [1]: [sum#37] +Results [11]: [c_last_name#16, c_first_name#15, s_store_name#2, ca_state#8, s_state#4, i_color#28, i_current_price#26, i_manager_id#30, i_units#29, i_size#27, sum#38] (41) Exchange -Input [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#37] -Arguments: hashpartitioning(c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [11]: [c_last_name#16, c_first_name#15, s_store_name#2, ca_state#8, s_state#4, i_color#28, i_current_price#26, i_manager_id#30, i_units#29, i_size#27, sum#38] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#2, ca_state#8, s_state#4, i_color#28, i_current_price#26, i_manager_id#30, i_units#29, i_size#27, 5), ENSURE_REQUIREMENTS, [id=#39] (42) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#37] -Keys [10]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#39] -Results [4]: [c_last_name#15, c_first_name#14, s_store_name#2, MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#39,17,2) AS netpaid#40] +Input [11]: [c_last_name#16, c_first_name#15, s_store_name#2, ca_state#8, s_state#4, i_color#28, i_current_price#26, i_manager_id#30, i_units#29, i_size#27, sum#38] +Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#2, ca_state#8, s_state#4, i_color#28, i_current_price#26, i_manager_id#30, i_units#29, i_size#27] +Functions [1]: [sum(UnscaledValue(ss_net_paid#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#23))#40] +Results [4]: [c_last_name#16, c_first_name#15, s_store_name#2, MakeDecimal(sum(UnscaledValue(ss_net_paid#23))#40,17,2) AS netpaid#41] (43) HashAggregate [codegen id : 10] -Input [4]: [c_last_name#15, c_first_name#14, s_store_name#2, netpaid#40] -Keys [3]: [c_last_name#15, c_first_name#14, s_store_name#2] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#15, c_first_name#14, s_store_name#2, sum#43, isEmpty#44] +Input [4]: [c_last_name#16, c_first_name#15, s_store_name#2, netpaid#41] +Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#2] +Functions [1]: [partial_sum(netpaid#41)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [5]: [c_last_name#16, c_first_name#15, s_store_name#2, sum#44, isEmpty#45] (44) Exchange -Input [5]: [c_last_name#15, c_first_name#14, s_store_name#2, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#15, c_first_name#14, s_store_name#2, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [5]: [c_last_name#16, c_first_name#15, s_store_name#2, sum#44, isEmpty#45] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#2, 5), ENSURE_REQUIREMENTS, [id=#46] (45) HashAggregate [codegen id : 11] -Input [5]: [c_last_name#15, c_first_name#14, s_store_name#2, sum#43, isEmpty#44] -Keys [3]: [c_last_name#15, c_first_name#14, s_store_name#2] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#46] -Results [4]: [c_last_name#15, c_first_name#14, s_store_name#2, sum(netpaid#40)#46 AS paid#47] +Input [5]: [c_last_name#16, c_first_name#15, s_store_name#2, sum#44, isEmpty#45] +Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#2] +Functions [1]: [sum(netpaid#41)] +Aggregate Attributes [1]: [sum(netpaid#41)#47] +Results [4]: [c_last_name#16, c_first_name#15, s_store_name#2, sum(netpaid#41)#47 AS paid#48] (46) Filter [codegen id : 11] -Input [4]: [c_last_name#15, c_first_name#14, s_store_name#2, paid#47] -Condition : (isnotnull(paid#47) AND (cast(paid#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) +Input [4]: [c_last_name#16, c_first_name#15, s_store_name#2, paid#48] +Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) (47) Exchange -Input [4]: [c_last_name#15, c_first_name#14, s_store_name#2, paid#47] -Arguments: rangepartitioning(c_last_name#15 ASC NULLS FIRST, c_first_name#14 ASC NULLS FIRST, s_store_name#2 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [4]: [c_last_name#16, c_first_name#15, s_store_name#2, paid#48] +Arguments: rangepartitioning(c_last_name#16 ASC NULLS FIRST, c_first_name#15 ASC NULLS FIRST, s_store_name#2 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#51] (48) Sort [codegen id : 12] -Input [4]: [c_last_name#15, c_first_name#14, s_store_name#2, paid#47] -Arguments: [c_last_name#15 ASC NULLS FIRST, c_first_name#14 ASC NULLS FIRST, s_store_name#2 ASC NULLS FIRST], true, 0 +Input [4]: [c_last_name#16, c_first_name#15, s_store_name#2, paid#48] +Arguments: [c_last_name#16 ASC NULLS FIRST, c_first_name#15 ASC NULLS FIRST, s_store_name#2 ASC NULLS FIRST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#48, [id=#49] -* HashAggregate (76) -+- Exchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- Exchange (72) - +- * HashAggregate (71) - +- * Project (70) - +- * SortMergeJoin Inner (69) - :- * Sort (66) - : +- Exchange (65) - : +- * Project (64) - : +- * SortMergeJoin Inner (63) - : :- * Sort (57) - : : +- Exchange (56) - : : +- * Project (55) - : : +- * BroadcastHashJoin Inner BuildLeft (54) - : : :- ReusedExchange (49) - : : +- * Project (53) - : : +- * Filter (52) - : : +- * ColumnarToRow (51) - : : +- Scan parquet default.store_sales (50) - : +- * Sort (62) - : +- Exchange (61) - : +- * Filter (60) - : +- * ColumnarToRow (59) - : +- Scan parquet default.item (58) - +- * Sort (68) - +- ReusedExchange (67) - - -(49) ReusedExchange [Reuses operator id: 17] -Output [7]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#54, c_customer_sk#55, c_first_name#56, c_last_name#57] - -(50) Scan parquet default.store_sales -Output [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +* HashAggregate (88) ++- Exchange (87) + +- * HashAggregate (86) + +- * HashAggregate (85) + +- Exchange (84) + +- * HashAggregate (83) + +- * Project (82) + +- * SortMergeJoin Inner (81) + :- * Sort (78) + : +- Exchange (77) + : +- * Project (76) + : +- * SortMergeJoin Inner (75) + : :- * Sort (69) + : : +- Exchange (68) + : : +- * Project (67) + : : +- * BroadcastHashJoin Inner BuildLeft (66) + : : :- BroadcastExchange (61) + : : : +- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildLeft (59) + : : : :- BroadcastExchange (55) + : : : : +- * Project (54) + : : : : +- * BroadcastHashJoin Inner BuildLeft (53) + : : : : :- ReusedExchange (49) + : : : : +- * Filter (52) + : : : : +- * ColumnarToRow (51) + : : : : +- Scan parquet default.customer_address (50) + : : : +- * Filter (58) + : : : +- * ColumnarToRow (57) + : : : +- Scan parquet default.customer (56) + : : +- * Project (65) + : : +- * Filter (64) + : : +- * ColumnarToRow (63) + : : +- Scan parquet default.store_sales (62) + : +- * Sort (74) + : +- Exchange (73) + : +- * Filter (72) + : +- * ColumnarToRow (71) + : +- Scan parquet default.item (70) + +- * Sort (80) + +- ReusedExchange (79) + + +(49) ReusedExchange [Reuses operator id: 5] +Output [4]: [s_store_sk#52, s_store_name#53, s_state#54, s_zip#55] + +(50) Scan parquet default.customer_address +Output [4]: [ca_address_sk#56, ca_state#57, ca_zip#58, ca_country#59] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] +ReadSchema: struct + +(51) ColumnarToRow +Input [4]: [ca_address_sk#56, ca_state#57, ca_zip#58, ca_country#59] + +(52) Filter +Input [4]: [ca_address_sk#56, ca_state#57, ca_zip#58, ca_country#59] +Condition : (((isnotnull(ca_address_sk#56) AND isnotnull(ca_country#59)) AND isnotnull(ca_zip#58)) AND isnotnull(upper(ca_country#59))) + +(53) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [s_zip#55] +Right keys [1]: [ca_zip#58] +Join condition: None + +(54) Project [codegen id : 2] +Output [6]: [s_store_sk#52, s_store_name#53, s_state#54, ca_address_sk#56, ca_state#57, upper(ca_country#59) AS upper(spark_catalog.default.customer_address.ca_country)#60] +Input [8]: [s_store_sk#52, s_store_name#53, s_state#54, s_zip#55, ca_address_sk#56, ca_state#57, ca_zip#58, ca_country#59] + +(55) BroadcastExchange +Input [6]: [s_store_sk#52, s_store_name#53, s_state#54, ca_address_sk#56, ca_state#57, upper(spark_catalog.default.customer_address.ca_country)#60] +Arguments: HashedRelationBroadcastMode(List(input[3, int, true], input[5, string, true]),false), [id=#61] + +(56) Scan parquet default.customer +Output [5]: [c_customer_sk#62, c_current_addr_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] +ReadSchema: struct + +(57) ColumnarToRow +Input [5]: [c_customer_sk#62, c_current_addr_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] + +(58) Filter +Input [5]: [c_customer_sk#62, c_current_addr_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Condition : ((isnotnull(c_customer_sk#62) AND isnotnull(c_current_addr_sk#63)) AND isnotnull(c_birth_country#66)) + +(59) BroadcastHashJoin [codegen id : 3] +Left keys [2]: [ca_address_sk#56, upper(spark_catalog.default.customer_address.ca_country)#60] +Right keys [2]: [c_current_addr_sk#63, c_birth_country#66] +Join condition: None + +(60) Project [codegen id : 3] +Output [7]: [s_store_sk#52, s_store_name#53, s_state#54, ca_state#57, c_customer_sk#62, c_first_name#64, c_last_name#65] +Input [11]: [s_store_sk#52, s_store_name#53, s_state#54, ca_address_sk#56, ca_state#57, upper(spark_catalog.default.customer_address.ca_country)#60, c_customer_sk#62, c_current_addr_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] + +(61) BroadcastExchange +Input [7]: [s_store_sk#52, s_store_name#53, s_state#54, ca_state#57, c_customer_sk#62, c_first_name#64, c_last_name#65] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#67] + +(62) Scan parquet default.store_sales +Output [6]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72, ss_sold_date_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(51) ColumnarToRow -Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] +(63) ColumnarToRow +Input [6]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72, ss_sold_date_sk#73] -(52) Filter -Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] -Condition : (((isnotnull(ss_ticket_number#61) AND isnotnull(ss_item_sk#58)) AND isnotnull(ss_store_sk#60)) AND isnotnull(ss_customer_sk#59)) +(64) Filter +Input [6]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72, ss_sold_date_sk#73] +Condition : (((isnotnull(ss_ticket_number#71) AND isnotnull(ss_item_sk#68)) AND isnotnull(ss_store_sk#70)) AND isnotnull(ss_customer_sk#69)) -(53) Project -Output [5]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62] -Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] +(65) Project +Output [5]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72] +Input [6]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72, ss_sold_date_sk#73] -(54) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [s_store_sk#51, c_customer_sk#55] -Right keys [2]: [ss_store_sk#60, ss_customer_sk#59] +(66) BroadcastHashJoin [codegen id : 4] +Left keys [2]: [s_store_sk#52, c_customer_sk#62] +Right keys [2]: [ss_store_sk#70, ss_customer_sk#69] Join condition: None -(55) Project [codegen id : 4] -Output [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] -Input [12]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#54, c_customer_sk#55, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62] +(67) Project [codegen id : 4] +Output [8]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72] +Input [12]: [s_store_sk#52, s_store_name#53, s_state#54, ca_state#57, c_customer_sk#62, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72] -(56) Exchange -Input [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] -Arguments: hashpartitioning(ss_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#64] +(68) Exchange +Input [8]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72] +Arguments: hashpartitioning(ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#74] -(57) Sort [codegen id : 5] -Input [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] -Arguments: [ss_item_sk#58 ASC NULLS FIRST], false, 0 +(69) Sort [codegen id : 5] +Input [8]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72] +Arguments: [ss_item_sk#68 ASC NULLS FIRST], false, 0 -(58) Scan parquet default.item -Output [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +(70) Scan parquet default.item +Output [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +(71) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] -(60) Filter [codegen id : 6] -Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Condition : isnotnull(i_item_sk#65) +(72) Filter [codegen id : 6] +Input [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +Condition : isnotnull(i_item_sk#75) -(61) Exchange -Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Arguments: hashpartitioning(i_item_sk#65, 5), ENSURE_REQUIREMENTS, [id=#71] +(73) Exchange +Input [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +Arguments: hashpartitioning(i_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] -(62) Sort [codegen id : 7] -Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Arguments: [i_item_sk#65 ASC NULLS FIRST], false, 0 +(74) Sort [codegen id : 7] +Input [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +Arguments: [i_item_sk#75 ASC NULLS FIRST], false, 0 -(63) SortMergeJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#58] -Right keys [1]: [i_item_sk#65] +(75) SortMergeJoin [codegen id : 8] +Left keys [1]: [ss_item_sk#68] +Right keys [1]: [i_item_sk#75] Join condition: None -(64) Project [codegen id : 8] -Output [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Input [14]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +(76) Project [codegen id : 8] +Output [13]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +Input [14]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] -(65) Exchange -Input [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Arguments: hashpartitioning(ss_ticket_number#61, ss_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#72] +(77) Exchange +Input [13]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +Arguments: hashpartitioning(ss_ticket_number#71, ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#82] -(66) Sort [codegen id : 9] -Input [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Arguments: [ss_ticket_number#61 ASC NULLS FIRST, ss_item_sk#58 ASC NULLS FIRST], false, 0 +(78) Sort [codegen id : 9] +Input [13]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +Arguments: [ss_ticket_number#71 ASC NULLS FIRST, ss_item_sk#68 ASC NULLS FIRST], false, 0 -(67) ReusedExchange [Reuses operator id: 36] -Output [2]: [sr_item_sk#73, sr_ticket_number#74] +(79) ReusedExchange [Reuses operator id: 36] +Output [2]: [sr_item_sk#83, sr_ticket_number#84] -(68) Sort [codegen id : 11] -Input [2]: [sr_item_sk#73, sr_ticket_number#74] -Arguments: [sr_ticket_number#74 ASC NULLS FIRST, sr_item_sk#73 ASC NULLS FIRST], false, 0 +(80) Sort [codegen id : 11] +Input [2]: [sr_item_sk#83, sr_ticket_number#84] +Arguments: [sr_ticket_number#84 ASC NULLS FIRST, sr_item_sk#83 ASC NULLS FIRST], false, 0 -(69) SortMergeJoin [codegen id : 12] -Left keys [2]: [ss_ticket_number#61, ss_item_sk#58] -Right keys [2]: [sr_ticket_number#74, sr_item_sk#73] +(81) SortMergeJoin [codegen id : 12] +Left keys [2]: [ss_ticket_number#71, ss_item_sk#68] +Right keys [2]: [sr_ticket_number#84, sr_item_sk#83] Join condition: None -(70) Project [codegen id : 12] -Output [11]: [ss_net_paid#62, s_store_name#52, s_state#53, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#56, c_last_name#57, ca_state#54] -Input [15]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, sr_item_sk#73, sr_ticket_number#74] - -(71) HashAggregate [codegen id : 12] -Input [11]: [ss_net_paid#62, s_store_name#52, s_state#53, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#56, c_last_name#57, ca_state#54] -Keys [10]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#62))] -Aggregate Attributes [1]: [sum#75] -Results [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] - -(72) Exchange -Input [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] -Arguments: hashpartitioning(c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, 5), ENSURE_REQUIREMENTS, [id=#77] - -(73) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] -Keys [10]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67] -Functions [1]: [sum(UnscaledValue(ss_net_paid#62))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#62))#78] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#62))#78,17,2) AS netpaid#40] - -(74) HashAggregate [codegen id : 13] -Input [1]: [netpaid#40] +(82) Project [codegen id : 12] +Output [11]: [ss_net_paid#72, s_store_name#53, s_state#54, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80, c_first_name#64, c_last_name#65, ca_state#57] +Input [15]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80, sr_item_sk#83, sr_ticket_number#84] + +(83) HashAggregate [codegen id : 12] +Input [11]: [ss_net_paid#72, s_store_name#53, s_state#54, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80, c_first_name#64, c_last_name#65, ca_state#57] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#53, ca_state#57, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#72))] +Aggregate Attributes [1]: [sum#85] +Results [11]: [c_last_name#65, c_first_name#64, s_store_name#53, ca_state#57, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77, sum#86] + +(84) Exchange +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#53, ca_state#57, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77, sum#86] +Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#53, ca_state#57, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77, 5), ENSURE_REQUIREMENTS, [id=#87] + +(85) HashAggregate [codegen id : 13] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#53, ca_state#57, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77, sum#86] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#53, ca_state#57, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77] +Functions [1]: [sum(UnscaledValue(ss_net_paid#72))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#72))#88] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#72))#88,17,2) AS netpaid#41] + +(86) HashAggregate [codegen id : 13] +Input [1]: [netpaid#41] Keys: [] -Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#79, count#80] -Results [2]: [sum#81, count#82] +Functions [1]: [partial_avg(netpaid#41)] +Aggregate Attributes [2]: [sum#89, count#90] +Results [2]: [sum#91, count#92] -(75) Exchange -Input [2]: [sum#81, count#82] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#83] +(87) Exchange +Input [2]: [sum#91, count#92] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] -(76) HashAggregate [codegen id : 14] -Input [2]: [sum#81, count#82] +(88) HashAggregate [codegen id : 14] +Input [2]: [sum#91, count#92] Keys: [] -Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#84] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#84)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#85] +Functions [1]: [avg(netpaid#41)] +Aggregate Attributes [1]: [avg(netpaid#41)#94] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#41)#94)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#95] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt index 4beebcbbe52ef..bdc45d61509bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt @@ -35,7 +35,25 @@ WholeStageCodegen (12) Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid] BroadcastHashJoin [s_store_sk,c_customer_sk,ss_store_sk,ss_customer_sk] InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] #5 + BroadcastExchange #14 + WholeStageCodegen (3) + Project [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] + BroadcastHashJoin [ca_address_sk,upper(spark_catalog.default.customer_address.ca_country),c_current_addr_sk,c_birth_country] + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (2) + Project [s_store_sk,s_store_name,s_state,ca_address_sk,ca_state,ca_country] + BroadcastHashJoin [s_zip,ca_zip] + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #7 + Filter [ca_address_sk,ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + Filter [c_customer_sk,c_current_addr_sk,c_birth_country] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow @@ -45,7 +63,7 @@ WholeStageCodegen (12) WholeStageCodegen (7) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #14 + Exchange [i_item_sk] #16 WholeStageCodegen (6) Filter [i_item_sk] ColumnarToRow @@ -82,7 +100,7 @@ WholeStageCodegen (12) BroadcastExchange #5 WholeStageCodegen (3) Project [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] - BroadcastHashJoin [ca_address_sk,ca_country,c_current_addr_sk,c_birth_country] + BroadcastHashJoin [ca_address_sk,upper(spark_catalog.default.customer_address.ca_country),c_current_addr_sk,c_birth_country] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt index ef15af68693a3..b205e0f39e8b2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt @@ -1,15 +1,15 @@ == Physical Plan == -* Sort (48) -+- Exchange (47) - +- * Filter (46) - +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) +* Sort (49) ++- Exchange (48) + +- * Filter (47) + +- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (33) : +- * BroadcastHashJoin Inner BuildRight (32) : :- * Project (27) @@ -43,10 +43,11 @@ : +- * Filter (30) : +- * ColumnarToRow (29) : +- Scan parquet default.customer (28) - +- BroadcastExchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.customer_address (34) + +- BroadcastExchange (38) + +- * Project (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.customer_address (34) (1) Scan parquet default.store_sales @@ -209,219 +210,246 @@ Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] Condition : ((isnotnull(ca_address_sk#31) AND isnotnull(ca_country#34)) AND isnotnull(ca_zip#33)) -(37) BroadcastExchange +(37) Project [codegen id : 8] +Output [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, upper(ca_country#34) AS upper(spark_catalog.default.customer_address.ca_country)#35] Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#35] -(38) BroadcastHashJoin [codegen id : 9] +(38) BroadcastExchange +Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, upper(spark_catalog.default.customer_address.ca_country)#35] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[3, string, true], input[2, string, true]),false), [id=#36] + +(39) BroadcastHashJoin [codegen id : 9] Left keys [3]: [c_current_addr_sk#26, c_birth_country#29, s_zip#16] -Right keys [3]: [ca_address_sk#31, upper(ca_country#34), ca_zip#33] +Right keys [3]: [ca_address_sk#31, upper(spark_catalog.default.customer_address.ca_country)#35, ca_zip#33] Join condition: None -(39) Project [codegen id : 9] +(40) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#27, c_last_name#28, ca_state#32] -Input [17]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29, ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] +Input [17]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29, ca_address_sk#31, ca_state#32, ca_zip#33, upper(spark_catalog.default.customer_address.ca_country)#35] -(40) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#27, c_last_name#28, ca_state#32] Keys [10]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#36] -Results [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#37] +Aggregate Attributes [1]: [sum#37] +Results [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#38] -(41) Exchange -Input [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#37] -Arguments: hashpartitioning(c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#38] +(42) Exchange +Input [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#38] +Arguments: hashpartitioning(c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#39] -(42) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#37] +(43) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#38] Keys [10]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#28, c_first_name#27, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#40] +Results [4]: [c_last_name#28, c_first_name#27, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#40,17,2) AS netpaid#41] -(43) HashAggregate [codegen id : 10] -Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, netpaid#40] +(44) HashAggregate [codegen id : 10] +Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, netpaid#41] Keys [3]: [c_last_name#28, c_first_name#27, s_store_name#13] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum#43, isEmpty#44] +Functions [1]: [partial_sum(netpaid#41)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum#44, isEmpty#45] -(44) Exchange -Input [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#28, c_first_name#27, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#45] +(45) Exchange +Input [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum#44, isEmpty#45] +Arguments: hashpartitioning(c_last_name#28, c_first_name#27, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#46] -(45) HashAggregate [codegen id : 11] -Input [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum#43, isEmpty#44] +(46) HashAggregate [codegen id : 11] +Input [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum#44, isEmpty#45] Keys [3]: [c_last_name#28, c_first_name#27, s_store_name#13] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#46] -Results [4]: [c_last_name#28, c_first_name#27, s_store_name#13, sum(netpaid#40)#46 AS paid#47] +Functions [1]: [sum(netpaid#41)] +Aggregate Attributes [1]: [sum(netpaid#41)#47] +Results [4]: [c_last_name#28, c_first_name#27, s_store_name#13, sum(netpaid#41)#47 AS paid#48] -(46) Filter [codegen id : 11] -Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#47] -Condition : (isnotnull(paid#47) AND (cast(paid#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) +(47) Filter [codegen id : 11] +Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#48] +Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) -(47) Exchange -Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#47] -Arguments: rangepartitioning(c_last_name#28 ASC NULLS FIRST, c_first_name#27 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#50] +(48) Exchange +Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#48] +Arguments: rangepartitioning(c_last_name#28 ASC NULLS FIRST, c_first_name#27 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#51] -(48) Sort [codegen id : 12] -Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#47] +(49) Sort [codegen id : 12] +Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#48] Arguments: [c_last_name#28 ASC NULLS FIRST, c_first_name#27 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#48, [id=#49] -* HashAggregate (75) -+- Exchange (74) - +- * HashAggregate (73) - +- * HashAggregate (72) - +- Exchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (57) - : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : :- * Project (54) - : : : : +- * SortMergeJoin Inner (53) - : : : : :- * Sort (50) - : : : : : +- ReusedExchange (49) - : : : : +- * Sort (52) - : : : : +- ReusedExchange (51) - : : : +- ReusedExchange (55) - : : +- BroadcastExchange (61) - : : +- * Filter (60) - : : +- * ColumnarToRow (59) - : : +- Scan parquet default.item (58) - : +- ReusedExchange (64) - +- ReusedExchange (67) - - -(49) ReusedExchange [Reuses operator id: 5] -Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] - -(50) Sort [codegen id : 2] -Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Arguments: [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST], false, 0 - -(51) ReusedExchange [Reuses operator id: 11] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] - -(52) Sort [codegen id : 4] -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST], false, 0 - -(53) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#54, ss_item_sk#51] -Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] +Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +* HashAggregate (80) ++- Exchange (79) + +- * HashAggregate (78) + +- * HashAggregate (77) + +- Exchange (76) + +- * HashAggregate (75) + +- * Project (74) + +- * BroadcastHashJoin Inner BuildRight (73) + :- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (58) + : : : +- * BroadcastHashJoin Inner BuildRight (57) + : : : :- * Project (55) + : : : : +- * SortMergeJoin Inner (54) + : : : : :- * Sort (51) + : : : : : +- ReusedExchange (50) + : : : : +- * Sort (53) + : : : : +- ReusedExchange (52) + : : : +- ReusedExchange (56) + : : +- BroadcastExchange (62) + : : +- * Filter (61) + : : +- * ColumnarToRow (60) + : : +- Scan parquet default.item (59) + : +- ReusedExchange (65) + +- BroadcastExchange (72) + +- * Project (71) + +- * Filter (70) + +- * ColumnarToRow (69) + +- Scan parquet default.customer_address (68) + + +(50) ReusedExchange [Reuses operator id: 5] +Output [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] + +(51) Sort [codegen id : 2] +Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] +Arguments: [ss_ticket_number#55 ASC NULLS FIRST, ss_item_sk#52 ASC NULLS FIRST], false, 0 + +(52) ReusedExchange [Reuses operator id: 11] +Output [2]: [sr_item_sk#57, sr_ticket_number#58] + +(53) Sort [codegen id : 4] +Input [2]: [sr_item_sk#57, sr_ticket_number#58] +Arguments: [sr_ticket_number#58 ASC NULLS FIRST, sr_item_sk#57 ASC NULLS FIRST], false, 0 + +(54) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#55, ss_item_sk#52] +Right keys [2]: [sr_ticket_number#58, sr_item_sk#57] Join condition: None -(54) Project [codegen id : 9] -Output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] +(55) Project [codegen id : 9] +Output [4]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56] +Input [7]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, sr_item_sk#57, sr_ticket_number#58] -(55) ReusedExchange [Reuses operator id: 19] -Output [4]: [s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] +(56) ReusedExchange [Reuses operator id: 19] +Output [4]: [s_store_sk#59, s_store_name#60, s_state#61, s_zip#62] -(56) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#53] -Right keys [1]: [s_store_sk#58] +(57) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#54] +Right keys [1]: [s_store_sk#59] Join condition: None -(57) Project [codegen id : 9] -Output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61] -Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] +(58) Project [codegen id : 9] +Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#60, s_state#61, s_zip#62] +Input [8]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56, s_store_sk#59, s_store_name#60, s_state#61, s_zip#62] -(58) Scan parquet default.item -Output [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +(59) Scan parquet default.item +Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +(60) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -(60) Filter [codegen id : 6] -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Condition : isnotnull(i_item_sk#62) +(61) Filter [codegen id : 6] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Condition : isnotnull(i_item_sk#63) -(61) BroadcastExchange -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] +(62) BroadcastExchange +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] -(62) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#51] -Right keys [1]: [i_item_sk#62] +(63) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#52] +Right keys [1]: [i_item_sk#63] Join condition: None -(63) Project [codegen id : 9] -Output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +(64) Project [codegen id : 9] +Output [10]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#60, s_state#61, s_zip#62, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Input [12]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#60, s_state#61, s_zip#62, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -(64) ReusedExchange [Reuses operator id: 31] -Output [5]: [c_customer_sk#69, c_current_addr_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +(65) ReusedExchange [Reuses operator id: 31] +Output [5]: [c_customer_sk#70, c_current_addr_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -(65) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#52] -Right keys [1]: [c_customer_sk#69] +(66) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#53] +Right keys [1]: [c_customer_sk#70] Join condition: None -(66) Project [codegen id : 9] -Output [13]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_current_addr_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] -Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_customer_sk#69, c_current_addr_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +(67) Project [codegen id : 9] +Output [13]: [ss_net_paid#56, s_store_name#60, s_state#61, s_zip#62, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_current_addr_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Input [15]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#60, s_state#61, s_zip#62, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#70, c_current_addr_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] + +(68) Scan parquet default.customer_address +Output [4]: [ca_address_sk#75, ca_state#76, ca_zip#77, ca_country#78] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] +ReadSchema: struct + +(69) ColumnarToRow [codegen id : 8] +Input [4]: [ca_address_sk#75, ca_state#76, ca_zip#77, ca_country#78] + +(70) Filter [codegen id : 8] +Input [4]: [ca_address_sk#75, ca_state#76, ca_zip#77, ca_country#78] +Condition : (((isnotnull(ca_address_sk#75) AND isnotnull(ca_country#78)) AND isnotnull(ca_zip#77)) AND isnotnull(upper(ca_country#78))) + +(71) Project [codegen id : 8] +Output [4]: [ca_address_sk#75, ca_state#76, ca_zip#77, upper(ca_country#78) AS upper(spark_catalog.default.customer_address.ca_country)#79] +Input [4]: [ca_address_sk#75, ca_state#76, ca_zip#77, ca_country#78] -(67) ReusedExchange [Reuses operator id: 37] -Output [4]: [ca_address_sk#74, ca_state#75, ca_zip#76, ca_country#77] +(72) BroadcastExchange +Input [4]: [ca_address_sk#75, ca_state#76, ca_zip#77, upper(spark_catalog.default.customer_address.ca_country)#79] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[3, string, true], input[2, string, true]),false), [id=#80] -(68) BroadcastHashJoin [codegen id : 9] -Left keys [3]: [c_current_addr_sk#70, c_birth_country#73, s_zip#61] -Right keys [3]: [ca_address_sk#74, upper(ca_country#77), ca_zip#76] +(73) BroadcastHashJoin [codegen id : 9] +Left keys [3]: [c_current_addr_sk#71, c_birth_country#74, s_zip#62] +Right keys [3]: [ca_address_sk#75, upper(spark_catalog.default.customer_address.ca_country)#79, ca_zip#77] Join condition: None -(69) Project [codegen id : 9] -Output [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#75] -Input [17]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_current_addr_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73, ca_address_sk#74, ca_state#75, ca_zip#76, ca_country#77] - -(70) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#75] -Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum#78] -Results [11]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#79] - -(71) Exchange -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#79] -Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#80] - -(72) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#79] -Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] -Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#81] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#81,17,2) AS netpaid#40] - -(73) HashAggregate [codegen id : 10] -Input [1]: [netpaid#40] +(74) Project [codegen id : 9] +Output [11]: [ss_net_paid#56, s_store_name#60, s_state#61, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#76] +Input [17]: [ss_net_paid#56, s_store_name#60, s_state#61, s_zip#62, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_current_addr_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74, ca_address_sk#75, ca_state#76, ca_zip#77, upper(spark_catalog.default.customer_address.ca_country)#79] + +(75) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#56, s_store_name#60, s_state#61, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#76] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#60, ca_state#76, s_state#61, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#56))] +Aggregate Attributes [1]: [sum#81] +Results [11]: [c_last_name#73, c_first_name#72, s_store_name#60, ca_state#76, s_state#61, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#82] + +(76) Exchange +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#60, ca_state#76, s_state#61, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#82] +Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#60, ca_state#76, s_state#61, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#83] + +(77) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#60, ca_state#76, s_state#61, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#82] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#60, ca_state#76, s_state#61, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [sum(UnscaledValue(ss_net_paid#56))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#56))#84] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#56))#84,17,2) AS netpaid#41] + +(78) HashAggregate [codegen id : 10] +Input [1]: [netpaid#41] Keys: [] -Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#82, count#83] -Results [2]: [sum#84, count#85] +Functions [1]: [partial_avg(netpaid#41)] +Aggregate Attributes [2]: [sum#85, count#86] +Results [2]: [sum#87, count#88] -(74) Exchange -Input [2]: [sum#84, count#85] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#86] +(79) Exchange +Input [2]: [sum#87, count#88] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#89] -(75) HashAggregate [codegen id : 11] -Input [2]: [sum#84, count#85] +(80) HashAggregate [codegen id : 11] +Input [2]: [sum#87, count#88] Keys: [] -Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#87] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#87)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#88] +Functions [1]: [avg(netpaid#41)] +Aggregate Attributes [1]: [avg(netpaid#41)#90] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#41)#90)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#91] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt index 0550ba1f05d58..d49112a3b65c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (12) WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,upper(spark_catalog.default.customer_address.ca_country),ca_zip] Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -48,7 +48,13 @@ WholeStageCodegen (12) InputAdapter ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 + BroadcastExchange #13 + WholeStageCodegen (8) + Project [ca_address_sk,ca_state,ca_zip,ca_country] + Filter [ca_address_sk,ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name,s_store_name] #2 @@ -60,7 +66,7 @@ WholeStageCodegen (12) WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,upper(spark_catalog.default.customer_address.ca_country),ca_zip] Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -116,7 +122,8 @@ WholeStageCodegen (12) InputAdapter BroadcastExchange #9 WholeStageCodegen (8) - Filter [ca_address_sk,ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + Project [ca_address_sk,ca_state,ca_zip,ca_country] + Filter [ca_address_sk,ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] 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 1e64471e096cb..db497ae40b481 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 @@ -223,61 +223,61 @@ Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_yea Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (rn#39 + 1) AS (v1_lag.rn + 1)#40] Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] (40) Exchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40, 5), ENSURE_REQUIREMENTS, [id=#41] (41) Sort [codegen id : 23] -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] +Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (v1_lag.rn + 1)#40 ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40] Join condition: None (43) Project [codegen id : 24] Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37] -Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] (44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] (45) Sort [codegen id : 33] -Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 (46) Window -Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] -Arguments: [rank(d_year#45, d_moy#46) windowspecdefinition(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#48], [i_category#41, i_brand#42, s_store_name#43, s_company_name#44], [d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST] +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Input [8]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47, rn#48] +Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (rn#49 - 1) AS (v1_lead.rn - 1)#50] +Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] (48) Exchange -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: hashpartitioning(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1), 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] +Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50, 5), ENSURE_REQUIREMENTS, [id=#51] (49) Sort [codegen id : 35] -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, (rn#48 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (v1_lead.rn - 1)#50 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] +Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50] Join condition: None (51) Project [codegen id : 36] -Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] -Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#52, sum_sales#48 AS nsum#53] +Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] (52) TakeOrderedAndProject -Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] +Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt index 5f64a22717270..f522534d62c80 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] WholeStageCodegen (36) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,s_store_name,s_company_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,s_store_name,s_company_name,rn] + Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #8 + Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] #8 WholeStageCodegen (22) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,s_store_name,s_company_name,rn] + Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #10 + Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] #10 WholeStageCodegen (34) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter 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 fcb93277bc76a..15c7c9cac4a94 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 @@ -196,53 +196,53 @@ Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_yea Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#26, i_brand#27, s_store_name#28, s_company_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] +Output [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, (rn#37 + 1) AS (v1_lag.rn + 1)#38] Input [8]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] (35) BroadcastExchange -Input [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [id=#38] +Input [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], input[5, int, false]),false), [id=#39] (36) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#24] -Right keys [5]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, (rn#37 + 1)] +Right keys [5]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, (v1_lag.rn + 1)#38] Join condition: None (37) Project [codegen id : 22] Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [15]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] +Input [15]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, (v1_lag.rn + 1)#38] (38) ReusedExchange [Reuses operator id: 31] -Output [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] +Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] (39) Sort [codegen id : 20] -Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, s_store_name#41 ASC NULLS FIRST, s_company_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 (40) Window -Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#39, i_brand#40, s_store_name#41, s_company_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] +Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#40, i_brand#41, s_store_name#42, s_company_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] -Input [8]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] +Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, (rn#47 - 1) AS (v1_lead.rn - 1)#48] +Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] (42) BroadcastExchange -Input [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [id=#47] +Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, (v1_lead.rn - 1)#48] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], input[5, int, false]),false), [id=#49] (43) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#24] -Right keys [5]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, (rn#46 - 1)] +Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (v1_lead.rn - 1)#48] Join condition: None (44) Project [codegen id : 22] -Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#48, sum_sales#45 AS nsum#49] -Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] +Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#50, sum_sales#46 AS nsum#51] +Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, (v1_lead.rn - 1)#48] (45) TakeOrderedAndProject -Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#48, nsum#49] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST], [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#48, nsum#49] +Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#50, nsum#51] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST], [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#50, nsum#51] ===== Subqueries ===== @@ -269,6 +269,6 @@ Condition : ((((d_year#11 = 1999) OR ((d_year#11 = 1998) AND (d_moy#12 = 12))) O (49) BroadcastExchange Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index 003a906be0dd9..4f2c0b0a78a06 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] WholeStageCodegen (22) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] Filter [avg_monthly_sales,sum_sales] InputAdapter 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 e7cd713d07cb4..0d87664dfb7a5 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 @@ -223,61 +223,61 @@ Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sale Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (rn#37 + 1) AS (v1_lag.rn + 1)#38] Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] (40) Exchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38, 5), ENSURE_REQUIREMENTS, [id=#39] (41) Sort [codegen id : 23] -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (v1_lag.rn + 1)#38 ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38] Join condition: None (43) Project [codegen id : 24] Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] (44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] (45) Sort [codegen id : 33] -Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 (46) Window -Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#39, i_brand#40, cc_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#45], [i_category#39, i_brand#40, cc_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] +Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (rn#46 - 1) AS (v1_lead.rn - 1)#47] +Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] (48) Exchange -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: hashpartitioning(i_category#39, i_brand#40, cc_name#41, (rn#45 - 1), 5), ENSURE_REQUIREMENTS, [id=#46] +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] +Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47, 5), ENSURE_REQUIREMENTS, [id=#48] (49) Sort [codegen id : 35] -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, (rn#45 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (v1_lead.rn - 1)#47 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] +Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47] Join condition: None (51) Project [codegen id : 36] -Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] -Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#49, sum_sales#45 AS nsum#50] +Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] (52) TakeOrderedAndProject -Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt index b464f558bbc1a..f5acc760c88f3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] WholeStageCodegen (36) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lead.rn - 1)] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lag.rn + 1)] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,cc_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,cc_name,rn] + Sort [i_category,i_brand,cc_name,(v1_lag.rn + 1)] InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #8 + Exchange [i_category,i_brand,cc_name,(v1_lag.rn + 1)] #8 WholeStageCodegen (22) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,cc_name,rn] + Sort [i_category,i_brand,cc_name,(v1_lead.rn - 1)] InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #10 + Exchange [i_category,i_brand,cc_name,(v1_lead.rn - 1)] #10 WholeStageCodegen (34) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter 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 1fb68bac3c7d8..59ef1e89952e9 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 @@ -196,53 +196,53 @@ Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sale Arguments: [rank(d_year#28, d_moy#29) windowspecdefinition(i_category#25, i_brand#26, cc_name#27, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#25, i_brand#26, cc_name#27], [d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] +Output [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, (rn#35 + 1) AS (v1_lag.rn + 1)#36] Input [7]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#33, rn#35] (35) BroadcastExchange -Input [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#36] +Input [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, (v1_lag.rn + 1)#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[4, int, false]),false), [id=#37] (36) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#23] -Right keys [4]: [i_category#25, i_brand#26, cc_name#27, (rn#35 + 1)] +Right keys [4]: [i_category#25, i_brand#26, cc_name#27, (v1_lag.rn + 1)#36] Join condition: None (37) Project [codegen id : 22] Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33] -Input [13]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] +Input [13]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, i_category#25, i_brand#26, cc_name#27, sum_sales#33, (v1_lag.rn + 1)#36] (38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] +Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] (39) Sort [codegen id : 20] -Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] -Arguments: [i_category#37 ASC NULLS FIRST, i_brand#38 ASC NULLS FIRST, cc_name#39 ASC NULLS FIRST, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST], false, 0 (40) Window -Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] -Arguments: [rank(d_year#40, d_moy#41) windowspecdefinition(i_category#37, i_brand#38, cc_name#39, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#43], [i_category#37, i_brand#38, cc_name#39], [d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST] +Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +Arguments: [rank(d_year#41, d_moy#42) windowspecdefinition(i_category#38, i_brand#39, cc_name#40, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] -Input [7]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42, rn#43] +Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, (rn#44 - 1) AS (v1_lead.rn - 1)#45] +Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] (42) BroadcastExchange -Input [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#44] +Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, (v1_lead.rn - 1)#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[4, int, false]),false), [id=#46] (43) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#23] -Right keys [4]: [i_category#37, i_brand#38, cc_name#39, (rn#43 - 1)] +Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (v1_lead.rn - 1)#45] Join condition: None (44) Project [codegen id : 22] -Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, sum_sales#33 AS psum#45, sum_sales#42 AS nsum#46] -Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33, i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] +Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, sum_sales#33 AS psum#47, sum_sales#43 AS nsum#48] +Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33, i_category#38, i_brand#39, cc_name#40, sum_sales#43, (v1_lead.rn - 1)#45] (45) TakeOrderedAndProject -Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#45, nsum#46] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#11 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#45, nsum#46] +Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#47, nsum#48] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#11 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#47, nsum#48] ===== Subqueries ===== @@ -269,6 +269,6 @@ Condition : ((((d_year#11 = 1999) OR ((d_year#11 = 1998) AND (d_moy#12 = 12))) O (49) BroadcastExchange Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index f099cef3b9d02..13080dde396c8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] WholeStageCodegen (22) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lead.rn - 1)] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lag.rn + 1)] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] Filter [avg_monthly_sales,sum_sales] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index c3997a7b32e2b..4091deb1911ed 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -1,20 +1,20 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * SortMergeJoin LeftOuter (65) - :- * Sort (58) - : +- Exchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * SortMergeJoin Inner (49) - : : :- * Sort (37) - : : : +- Exchange (36) - : : : +- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) +TakeOrderedAndProject (80) ++- * HashAggregate (79) + +- Exchange (78) + +- * HashAggregate (77) + +- * Project (76) + +- * SortMergeJoin LeftOuter (75) + :- * Sort (68) + : +- Exchange (67) + : +- * Project (66) + : +- * BroadcastHashJoin LeftOuter BuildRight (65) + : :- * Project (60) + : : +- * SortMergeJoin Inner (59) + : : :- * Sort (47) + : : : +- Exchange (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) : : : :- * Project (32) : : : : +- * SortMergeJoin Inner (31) : : : : :- * Sort (25) @@ -47,35 +47,45 @@ TakeOrderedAndProject (70) : : : : +- * Filter (28) : : : : +- * ColumnarToRow (27) : : : : +- Scan parquet default.item (26) - : : : +- ReusedExchange (33) - : : +- * Sort (48) - : : +- Exchange (47) - : : +- * Project (46) - : : +- * BroadcastHashJoin Inner BuildRight (45) - : : :- * Filter (40) - : : : +- * ColumnarToRow (39) - : : : +- Scan parquet default.inventory (38) - : : +- BroadcastExchange (44) - : : +- * Filter (43) - : : +- * ColumnarToRow (42) - : : +- Scan parquet default.warehouse (41) - : +- BroadcastExchange (54) - : +- * Filter (53) - : +- * ColumnarToRow (52) - : +- Scan parquet default.promotion (51) - +- * Sort (64) - +- Exchange (63) - +- * Project (62) - +- * Filter (61) - +- * ColumnarToRow (60) - +- Scan parquet default.catalog_returns (59) + : : : +- BroadcastExchange (43) + : : : +- * Project (42) + : : : +- * BroadcastHashJoin Inner BuildLeft (41) + : : : :- BroadcastExchange (37) + : : : : +- * Project (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.date_dim (33) + : : : +- * Filter (40) + : : : +- * ColumnarToRow (39) + : : : +- Scan parquet default.date_dim (38) + : : +- * Sort (58) + : : +- Exchange (57) + : : +- * Project (56) + : : +- * BroadcastHashJoin Inner BuildRight (55) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.inventory (48) + : : +- BroadcastExchange (54) + : : +- * Filter (53) + : : +- * ColumnarToRow (52) + : : +- Scan parquet default.warehouse (51) + : +- BroadcastExchange (64) + : +- * Filter (63) + : +- * ColumnarToRow (62) + : +- Scan parquet default.promotion (61) + +- * Sort (74) + +- Exchange (73) + +- * Project (72) + +- * Filter (71) + +- * ColumnarToRow (70) + +- Scan parquet default.catalog_returns (69) (1) Scan parquet default.catalog_sales Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] ReadSchema: struct @@ -87,367 +97,347 @@ Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_s Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) (4) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#10, hd_buy_potential#11] +Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [hd_demo_sk#10, hd_buy_potential#11] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] -Input [2]: [hd_demo_sk#10, hd_buy_potential#11] -Condition : ((isnotnull(hd_buy_potential#11) AND (hd_buy_potential#11 = 1001-5000 )) AND isnotnull(hd_demo_sk#10)) +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000 )) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] -Output [1]: [hd_demo_sk#10] -Input [2]: [hd_demo_sk#10, hd_buy_potential#11] +Output [1]: [hd_demo_sk#9] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (8) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +Input [1]: [hd_demo_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (9) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#10] +Right keys [1]: [hd_demo_sk#9] Join condition: None (10) Project [codegen id : 4] Output [7]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Input [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, hd_demo_sk#10] +Input [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, hd_demo_sk#9] (11) Scan parquet default.customer_demographics -Output [2]: [cd_demo_sk#13, cd_marital_status#14] +Output [2]: [cd_demo_sk#12, cd_marital_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_marital_status), EqualTo(cd_marital_status,M), IsNotNull(cd_demo_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [2]: [cd_demo_sk#13, cd_marital_status#14] +Input [2]: [cd_demo_sk#12, cd_marital_status#13] (13) Filter [codegen id : 2] -Input [2]: [cd_demo_sk#13, cd_marital_status#14] -Condition : ((isnotnull(cd_marital_status#14) AND (cd_marital_status#14 = M)) AND isnotnull(cd_demo_sk#13)) +Input [2]: [cd_demo_sk#12, cd_marital_status#13] +Condition : ((isnotnull(cd_marital_status#13) AND (cd_marital_status#13 = M)) AND isnotnull(cd_demo_sk#12)) (14) Project [codegen id : 2] -Output [1]: [cd_demo_sk#13] -Input [2]: [cd_demo_sk#13, cd_marital_status#14] +Output [1]: [cd_demo_sk#12] +Input [2]: [cd_demo_sk#12, cd_marital_status#13] (15) BroadcastExchange -Input [1]: [cd_demo_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +Input [1]: [cd_demo_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (16) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#13] +Right keys [1]: [cd_demo_sk#12] Join condition: None (17) Project [codegen id : 4] Output [6]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, cd_demo_sk#13] +Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, cd_demo_sk#12] (18) Scan parquet default.date_dim -Output [2]: [d_date_sk#16, d_date#17] +Output [2]: [d_date_sk#15, d_date#16] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#16, d_date#17] +Input [2]: [d_date_sk#15, d_date#16] (20) Filter [codegen id : 3] -Input [2]: [d_date_sk#16, d_date#17] -Condition : (isnotnull(d_date#17) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#15, d_date#16] +Condition : (isnotnull(d_date#16) AND isnotnull(d_date_sk#15)) (21) BroadcastExchange -Input [2]: [d_date_sk#16, d_date#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#18] +Input [2]: [d_date_sk#15, d_date#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (22) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#15] Join condition: None (23) Project [codegen id : 4] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17] -Input [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date_sk#16, d_date#17] +Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16] +Input [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date_sk#15, d_date#16] (24) Exchange -Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17] -Arguments: hashpartitioning(cs_item_sk#4, 5), ENSURE_REQUIREMENTS, [id=#19] +Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16] +Arguments: hashpartitioning(cs_item_sk#4, 5), ENSURE_REQUIREMENTS, [id=#18] (25) Sort [codegen id : 5] -Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17] +Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16] Arguments: [cs_item_sk#4 ASC NULLS FIRST], false, 0 (26) Scan parquet default.item -Output [2]: [i_item_sk#20, i_item_desc#21] +Output [2]: [i_item_sk#19, i_item_desc#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#20, i_item_desc#21] +Input [2]: [i_item_sk#19, i_item_desc#20] (28) Filter [codegen id : 6] -Input [2]: [i_item_sk#20, i_item_desc#21] -Condition : isnotnull(i_item_sk#20) +Input [2]: [i_item_sk#19, i_item_desc#20] +Condition : isnotnull(i_item_sk#19) (29) Exchange -Input [2]: [i_item_sk#20, i_item_desc#21] -Arguments: hashpartitioning(i_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [2]: [i_item_sk#19, i_item_desc#20] +Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 7] -Input [2]: [i_item_sk#20, i_item_desc#21] -Arguments: [i_item_sk#20 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#19, i_item_desc#20] +Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 (31) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#20] +Right keys [1]: [i_item_sk#19] Join condition: None (32) Project [codegen id : 10] -Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21] -Input [8]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_sk#20, i_item_desc#21] +Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_desc#20] +Input [8]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_sk#19, i_item_desc#20] -(33) ReusedExchange [Reuses operator id: 81] -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] +(33) Scan parquet default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] +ReadSchema: struct + +(34) ColumnarToRow [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] + +(35) Filter [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(34) BroadcastHashJoin [codegen id : 10] +(36) Project [codegen id : 8] +Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] + +(37) BroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] + +(38) Scan parquet default.date_dim +Output [2]: [d_date_sk#27, d_week_seq#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(39) ColumnarToRow +Input [2]: [d_date_sk#27, d_week_seq#28] + +(40) Filter +Input [2]: [d_date_sk#27, d_week_seq#28] +Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) + +(41) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [d_week_seq#24] +Right keys [1]: [d_week_seq#28] +Join condition: None + +(42) Project [codegen id : 9] +Output [4]: [d_date_sk#22, d_week_seq#24, d_date_sk#27, d_date#23 + 5 days AS d1.d_date + INTERVAL '5 days'#29] +Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] + +(43) BroadcastExchange +Input [4]: [d_date_sk#22, d_week_seq#24, d_date_sk#27, d1.d_date + INTERVAL '5 days'#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] + +(44) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#23] -Join condition: (d_date#17 > d_date#24 + 5 days) +Right keys [1]: [d_date_sk#22] +Join condition: (d_date#16 > d1.d_date + INTERVAL '5 days'#29) -(35) Project [codegen id : 10] -Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26] -Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21, d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] +(45) Project [codegen id : 10] +Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_desc#20, d_date_sk#22, d_week_seq#24, d_date_sk#27, d1.d_date + INTERVAL '5 days'#29] -(36) Exchange -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26] -Arguments: hashpartitioning(cs_item_sk#4, d_date_sk#26, 5), ENSURE_REQUIREMENTS, [id=#27] +(46) Exchange +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: hashpartitioning(cs_item_sk#4, d_date_sk#27, 5), ENSURE_REQUIREMENTS, [id=#31] -(37) Sort [codegen id : 11] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26] -Arguments: [cs_item_sk#4 ASC NULLS FIRST, d_date_sk#26 ASC NULLS FIRST], false, 0 +(47) Sort [codegen id : 11] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: [cs_item_sk#4 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.inventory -Output [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31] +(48) Scan parquet default.inventory +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#31), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(inv_date_sk#35), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 13] -Input [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31] +(49) ColumnarToRow [codegen id : 13] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] -(40) Filter [codegen id : 13] -Input [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31] -Condition : ((isnotnull(inv_quantity_on_hand#30) AND isnotnull(inv_item_sk#28)) AND isnotnull(inv_warehouse_sk#29)) +(50) Filter [codegen id : 13] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Condition : ((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) -(41) Scan parquet default.warehouse -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] +(51) Scan parquet default.warehouse +Output [2]: [w_warehouse_sk#36, w_warehouse_name#37] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 12] -Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] +(52) ColumnarToRow [codegen id : 12] +Input [2]: [w_warehouse_sk#36, w_warehouse_name#37] -(43) Filter [codegen id : 12] -Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] -Condition : isnotnull(w_warehouse_sk#32) +(53) Filter [codegen id : 12] +Input [2]: [w_warehouse_sk#36, w_warehouse_name#37] +Condition : isnotnull(w_warehouse_sk#36) -(44) BroadcastExchange -Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +(54) BroadcastExchange +Input [2]: [w_warehouse_sk#36, w_warehouse_name#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] -(45) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [inv_warehouse_sk#29] -Right keys [1]: [w_warehouse_sk#32] +(55) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [inv_warehouse_sk#33] +Right keys [1]: [w_warehouse_sk#36] Join condition: None -(46) Project [codegen id : 13] -Output [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] -Input [6]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_sk#32, w_warehouse_name#33] +(56) Project [codegen id : 13] +Output [4]: [inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] +Input [6]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_sk#36, w_warehouse_name#37] -(47) Exchange -Input [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] -Arguments: hashpartitioning(inv_item_sk#28, inv_date_sk#31, 5), ENSURE_REQUIREMENTS, [id=#35] +(57) Exchange +Input [4]: [inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] +Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#35, 5), ENSURE_REQUIREMENTS, [id=#39] -(48) Sort [codegen id : 14] -Input [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] -Arguments: [inv_item_sk#28 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 +(58) Sort [codegen id : 14] +Input [4]: [inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#35 ASC NULLS FIRST], false, 0 -(49) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#4, d_date_sk#26] -Right keys [2]: [inv_item_sk#28, inv_date_sk#31] -Join condition: (inv_quantity_on_hand#30 < cs_quantity#7) +(59) SortMergeJoin [codegen id : 16] +Left keys [2]: [cs_item_sk#4, d_date_sk#27] +Right keys [2]: [inv_item_sk#32, inv_date_sk#35] +Join condition: (inv_quantity_on_hand#34 < cs_quantity#7) -(50) Project [codegen id : 16] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26, inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] +(60) Project [codegen id : 16] +Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#24] +Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] -(51) Scan parquet default.promotion -Output [1]: [p_promo_sk#36] +(61) Scan parquet default.promotion +Output [1]: [p_promo_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 15] -Input [1]: [p_promo_sk#36] +(62) ColumnarToRow [codegen id : 15] +Input [1]: [p_promo_sk#40] -(53) Filter [codegen id : 15] -Input [1]: [p_promo_sk#36] -Condition : isnotnull(p_promo_sk#36) +(63) Filter [codegen id : 15] +Input [1]: [p_promo_sk#40] +Condition : isnotnull(p_promo_sk#40) -(54) BroadcastExchange -Input [1]: [p_promo_sk#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] +(64) BroadcastExchange +Input [1]: [p_promo_sk#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#41] -(55) BroadcastHashJoin [codegen id : 16] +(65) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#36] +Right keys [1]: [p_promo_sk#40] Join condition: None -(56) Project [codegen id : 16] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25, p_promo_sk#36] +(66) Project [codegen id : 16] +Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#24, p_promo_sk#40] -(57) Exchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#38] +(67) Exchange +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#24] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#42] -(58) Sort [codegen id : 17] -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] +(68) Sort [codegen id : 17] +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#24] Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 -(59) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +(69) Scan parquet default.catalog_returns +Output [3]: [cr_item_sk#43, cr_order_number#44, cr_returned_date_sk#45] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(60) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +(70) ColumnarToRow [codegen id : 18] +Input [3]: [cr_item_sk#43, cr_order_number#44, cr_returned_date_sk#45] -(61) Filter [codegen id : 18] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] -Condition : (isnotnull(cr_item_sk#39) AND isnotnull(cr_order_number#40)) +(71) Filter [codegen id : 18] +Input [3]: [cr_item_sk#43, cr_order_number#44, cr_returned_date_sk#45] +Condition : (isnotnull(cr_item_sk#43) AND isnotnull(cr_order_number#44)) -(62) Project [codegen id : 18] -Output [2]: [cr_item_sk#39, cr_order_number#40] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +(72) Project [codegen id : 18] +Output [2]: [cr_item_sk#43, cr_order_number#44] +Input [3]: [cr_item_sk#43, cr_order_number#44, cr_returned_date_sk#45] -(63) Exchange -Input [2]: [cr_item_sk#39, cr_order_number#40] -Arguments: hashpartitioning(cr_item_sk#39, cr_order_number#40, 5), ENSURE_REQUIREMENTS, [id=#42] +(73) Exchange +Input [2]: [cr_item_sk#43, cr_order_number#44] +Arguments: hashpartitioning(cr_item_sk#43, cr_order_number#44, 5), ENSURE_REQUIREMENTS, [id=#46] -(64) Sort [codegen id : 19] -Input [2]: [cr_item_sk#39, cr_order_number#40] -Arguments: [cr_item_sk#39 ASC NULLS FIRST, cr_order_number#40 ASC NULLS FIRST], false, 0 +(74) Sort [codegen id : 19] +Input [2]: [cr_item_sk#43, cr_order_number#44] +Arguments: [cr_item_sk#43 ASC NULLS FIRST, cr_order_number#44 ASC NULLS FIRST], false, 0 -(65) SortMergeJoin [codegen id : 20] +(75) SortMergeJoin [codegen id : 20] Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#39, cr_order_number#40] +Right keys [2]: [cr_item_sk#43, cr_order_number#44] Join condition: None -(66) Project [codegen id : 20] -Output [3]: [w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25, cr_item_sk#39, cr_order_number#40] +(76) Project [codegen id : 20] +Output [3]: [w_warehouse_name#37, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#24, cr_item_sk#43, cr_order_number#44] -(67) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Keys [3]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25] +(77) HashAggregate [codegen id : 20] +Input [3]: [w_warehouse_name#37, i_item_desc#20, d_week_seq#24] +Keys [3]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#43] -Results [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44] +Aggregate Attributes [1]: [count#47] +Results [4]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24, count#48] -(68) Exchange -Input [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44] -Arguments: hashpartitioning(i_item_desc#21, w_warehouse_name#33, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#45] +(78) Exchange +Input [4]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24, count#48] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#37, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [id=#49] -(69) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44] -Keys [3]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25] +(79) HashAggregate [codegen id : 21] +Input [4]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24, count#48] +Keys [3]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#46] -Results [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count(1)#46 AS no_promo#47, count(1)#46 AS promo#48, count(1)#46 AS total_cnt#49] - -(70) TakeOrderedAndProject -Input [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] -Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (81) -+- * Project (80) - +- * BroadcastHashJoin Inner BuildLeft (79) - :- BroadcastExchange (75) - : +- * Project (74) - : +- * Filter (73) - : +- * ColumnarToRow (72) - : +- Scan parquet default.date_dim (71) - +- * Filter (78) - +- * ColumnarToRow (77) - +- Scan parquet default.date_dim (76) - - -(71) Scan parquet default.date_dim -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(72) ColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] - -(73) Filter [codegen id : 1] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] -Condition : ((((isnotnull(d_year#50) AND (d_year#50 = 2001)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) - -(74) Project [codegen id : 1] -Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] - -(75) BroadcastExchange -Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#51] - -(76) Scan parquet default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(77) ColumnarToRow -Input [2]: [d_date_sk#26, d_week_seq#52] - -(78) Filter -Input [2]: [d_date_sk#26, d_week_seq#52] -Condition : (isnotnull(d_week_seq#52) AND isnotnull(d_date_sk#26)) - -(79) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#25] -Right keys [1]: [d_week_seq#52] -Join condition: None - -(80) Project [codegen id : 2] -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] -Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#52] - -(81) BroadcastExchange -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] +Aggregate Attributes [1]: [count(1)#50] +Results [6]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24, count(1)#50 AS no_promo#51, count(1)#50 AS promo#52, count(1)#50 AS total_cnt#53] +(80) TakeOrderedAndProject +Input [6]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#24, no_promo#51, promo#52, total_cnt#53] +Arguments: 100, [total_cnt#53 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#37 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#37, d_week_seq#24, no_promo#51, promo#52, total_cnt#53] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt index e838025a71db8..6f6786c9eee3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Exchange [cs_item_sk,d_date_sk] #3 WholeStageCodegen (10) Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d1.d_date + INTERVAL '5 days'] Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date,i_item_desc] SortMergeJoin [cs_item_sk,i_item_sk] InputAdapter @@ -43,25 +43,8 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (2) - Project [d_date_sk,d_date,d_week_seq,d_date_sk] - BroadcastHashJoin [d_week_seq,d_week_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #7 + BroadcastExchange #5 WholeStageCodegen (1) Project [hd_demo_sk] Filter [hd_buy_potential,hd_demo_sk] @@ -69,7 +52,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #8 + BroadcastExchange #6 WholeStageCodegen (2) Project [cd_demo_sk] Filter [cd_marital_status,cd_demo_sk] @@ -77,7 +60,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - BroadcastExchange #9 + BroadcastExchange #7 WholeStageCodegen (3) Filter [d_date,d_date_sk] ColumnarToRow @@ -87,14 +70,29 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom WholeStageCodegen (7) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #10 + Exchange [i_item_sk] #8 WholeStageCodegen (6) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #5 + BroadcastExchange #9 + WholeStageCodegen (9) + Project [d_date_sk,d_week_seq,d_date_sk,d_date] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter WholeStageCodegen (14) Sort [inv_item_sk,inv_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index 7654b76d7709b..c921631470ff5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -266,127 +266,127 @@ Right keys [2]: [d_week_seq#31, d_date_sk#30] Join condition: None (44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_date#28, d_week_seq#29] +Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29, d_date#28 + 5 days AS d1.d_date + INTERVAL '5 days'#33] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#16, i_item_desc#19, d_date#28, d_week_seq#29, d_date_sk#30, d_week_seq#31] (45) Scan parquet default.date_dim -Output [2]: [d_date_sk#33, d_date#34] +Output [2]: [d_date_sk#34, d_date#35] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (46) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#33, d_date#34] +Input [2]: [d_date_sk#34, d_date#35] (47) Filter [codegen id : 8] -Input [2]: [d_date_sk#33, d_date#34] -Condition : (isnotnull(d_date#34) AND isnotnull(d_date_sk#33)) +Input [2]: [d_date_sk#34, d_date#35] +Condition : (isnotnull(d_date#35) AND isnotnull(d_date_sk#34)) (48) BroadcastExchange -Input [2]: [d_date_sk#33, d_date#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#35] +Input [2]: [d_date_sk#34, d_date#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] (49) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#33] -Join condition: (d_date#34 > d_date#28 + 5 days) +Right keys [1]: [d_date_sk#34] +Join condition: (d_date#35 > d1.d_date + INTERVAL '5 days'#33) (50) Project [codegen id : 10] Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_date#28, d_week_seq#29, d_date_sk#33, d_date#34] +Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29, d1.d_date + INTERVAL '5 days'#33, d_date_sk#34, d_date#35] (51) Scan parquet default.promotion -Output [1]: [p_promo_sk#36] +Output [1]: [p_promo_sk#37] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#36] +Input [1]: [p_promo_sk#37] (53) Filter [codegen id : 9] -Input [1]: [p_promo_sk#36] -Condition : isnotnull(p_promo_sk#36) +Input [1]: [p_promo_sk#37] +Condition : isnotnull(p_promo_sk#37) (54) BroadcastExchange -Input [1]: [p_promo_sk#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] +Input [1]: [p_promo_sk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] (55) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#36] +Right keys [1]: [p_promo_sk#37] Join condition: None (56) Project [codegen id : 10] Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29, p_promo_sk#36] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29, p_promo_sk#37] (57) Exchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#38] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#39] (58) Sort [codegen id : 11] Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29] Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 (59) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +Output [3]: [cr_item_sk#40, cr_order_number#41, cr_returned_date_sk#42] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct (60) ColumnarToRow [codegen id : 12] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +Input [3]: [cr_item_sk#40, cr_order_number#41, cr_returned_date_sk#42] (61) Filter [codegen id : 12] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] -Condition : (isnotnull(cr_item_sk#39) AND isnotnull(cr_order_number#40)) +Input [3]: [cr_item_sk#40, cr_order_number#41, cr_returned_date_sk#42] +Condition : (isnotnull(cr_item_sk#40) AND isnotnull(cr_order_number#41)) (62) Project [codegen id : 12] -Output [2]: [cr_item_sk#39, cr_order_number#40] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +Output [2]: [cr_item_sk#40, cr_order_number#41] +Input [3]: [cr_item_sk#40, cr_order_number#41, cr_returned_date_sk#42] (63) Exchange -Input [2]: [cr_item_sk#39, cr_order_number#40] -Arguments: hashpartitioning(cr_item_sk#39, cr_order_number#40, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [2]: [cr_item_sk#40, cr_order_number#41] +Arguments: hashpartitioning(cr_item_sk#40, cr_order_number#41, 5), ENSURE_REQUIREMENTS, [id=#43] (64) Sort [codegen id : 13] -Input [2]: [cr_item_sk#39, cr_order_number#40] -Arguments: [cr_item_sk#39 ASC NULLS FIRST, cr_order_number#40 ASC NULLS FIRST], false, 0 +Input [2]: [cr_item_sk#40, cr_order_number#41] +Arguments: [cr_item_sk#40 ASC NULLS FIRST, cr_order_number#41 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 14] Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#39, cr_order_number#40] +Right keys [2]: [cr_item_sk#40, cr_order_number#41] Join condition: None (66) Project [codegen id : 14] Output [3]: [w_warehouse_name#16, i_item_desc#19, d_week_seq#29] -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29, cr_item_sk#39, cr_order_number#40] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#16, i_item_desc#19, d_week_seq#29, cr_item_sk#40, cr_order_number#41] (67) HashAggregate [codegen id : 14] Input [3]: [w_warehouse_name#16, i_item_desc#19, d_week_seq#29] Keys [3]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#43] -Results [4]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count#44] +Aggregate Attributes [1]: [count#44] +Results [4]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count#45] (68) Exchange -Input [4]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count#44] -Arguments: hashpartitioning(i_item_desc#19, w_warehouse_name#16, d_week_seq#29, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [4]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count#45] +Arguments: hashpartitioning(i_item_desc#19, w_warehouse_name#16, d_week_seq#29, 5), ENSURE_REQUIREMENTS, [id=#46] (69) HashAggregate [codegen id : 15] -Input [4]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count#44] +Input [4]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count#45] Keys [3]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#46] -Results [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count(1)#46 AS no_promo#47, count(1)#46 AS promo#48, count(1)#46 AS total_cnt#49] +Aggregate Attributes [1]: [count(1)#47] +Results [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count(1)#47 AS no_promo#48, count(1)#47 AS promo#49, count(1)#47 AS total_cnt#50] (70) TakeOrderedAndProject -Input [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49] -Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#19 ASC NULLS FIRST, w_warehouse_name#16 ASC NULLS FIRST, d_week_seq#29 ASC NULLS FIRST], [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49] +Input [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#48, promo#49, total_cnt#50] +Arguments: 100, [total_cnt#50 DESC NULLS LAST, i_item_desc#19 ASC NULLS FIRST, w_warehouse_name#16 ASC NULLS FIRST, d_week_seq#29 ASC NULLS FIRST], [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#48, promo#49, total_cnt#50] ===== Subqueries ===== @@ -399,25 +399,25 @@ BroadcastExchange (75) (71) Scan parquet default.date_dim -Output [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#50] +Output [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#51] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (72) ColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#50] +Input [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#51] (73) Filter [codegen id : 1] -Input [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#50] -Condition : ((((isnotnull(d_year#50) AND (d_year#50 = 2001)) AND isnotnull(d_date_sk#27)) AND isnotnull(d_week_seq#29)) AND isnotnull(d_date#28)) +Input [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#51] +Condition : ((((isnotnull(d_year#51) AND (d_year#51 = 2001)) AND isnotnull(d_date_sk#27)) AND isnotnull(d_week_seq#29)) AND isnotnull(d_date#28)) (74) Project [codegen id : 1] Output [3]: [d_date_sk#27, d_date#28, d_week_seq#29] -Input [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#50] +Input [4]: [d_date_sk#27, d_date#28, d_week_seq#29, d_year#51] (75) BroadcastExchange Input [3]: [d_date_sk#27, d_date#28, d_week_seq#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt index 7968b1cb84729..e684dba8c697c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d1.d_date + INTERVAL '5 days'] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,d_date] BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt index fea0bcbbef17e..67d964ee31184 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt @@ -350,94 +350,94 @@ Right keys [1]: [customer_id#56] Join condition: None (60) Project [codegen id : 26] -Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] +Output [6]: [customer_id#17, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#57, CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#38) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#59] Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] (61) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] +Output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#23)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] +Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] (63) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] -Condition : isnotnull(ws_bill_customer_sk#59) +Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +Condition : isnotnull(ws_bill_customer_sk#60) (64) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#62, d_year#63] +Output [2]: [d_date_sk#63, d_year#64] (65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#61] -Right keys [1]: [d_date_sk#62] +Left keys [1]: [ws_sold_date_sk#62] +Right keys [1]: [d_date_sk#63] Join condition: None (66) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63] -Input [5]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61, d_date_sk#62, d_year#63] +Output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64] +Input [5]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62, d_date_sk#63, d_year#64] (67) Exchange -Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63] -Arguments: hashpartitioning(ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64] +Arguments: hashpartitioning(ws_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (68) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63] -Arguments: [ws_bill_customer_sk#59 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64] +Arguments: [ws_bill_customer_sk#60 ASC NULLS FIRST], false, 0 (69) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#65, c_customer_id#66, c_first_name#67, c_last_name#68] +Output [4]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69] (70) Sort [codegen id : 31] -Input [4]: [c_customer_sk#65, c_customer_id#66, c_first_name#67, c_last_name#68] -Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69] +Arguments: [c_customer_sk#66 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#59] -Right keys [1]: [c_customer_sk#65] +Left keys [1]: [ws_bill_customer_sk#60] +Right keys [1]: [c_customer_sk#66] Join condition: None (72) Project [codegen id : 32] -Output [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, ws_net_paid#60, d_year#63] -Input [7]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63, c_customer_sk#65, c_customer_id#66, c_first_name#67, c_last_name#68] +Output [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, ws_net_paid#61, d_year#64] +Input [7]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64, c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69] (73) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, ws_net_paid#60, d_year#63] -Keys [4]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#60))] -Aggregate Attributes [1]: [sum#69] -Results [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, sum#70] +Input [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, ws_net_paid#61, d_year#64] +Keys [4]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#61))] +Aggregate Attributes [1]: [sum#70] +Results [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, sum#71] (74) Exchange -Input [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, sum#70] -Arguments: hashpartitioning(c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, sum#71] +Arguments: hashpartitioning(c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, 5), ENSURE_REQUIREMENTS, [id=#72] (75) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, sum#70] -Keys [4]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63] -Functions [1]: [sum(UnscaledValue(ws_net_paid#60))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#60))#72] -Results [2]: [c_customer_id#66 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#60))#72,17,2) AS year_total#74] +Input [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, sum#71] +Keys [4]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64] +Functions [1]: [sum(UnscaledValue(ws_net_paid#61))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#61))#73] +Results [2]: [c_customer_id#67 AS customer_id#74, MakeDecimal(sum(UnscaledValue(ws_net_paid#61))#73,17,2) AS year_total#75] (76) Exchange -Input [2]: [customer_id#73, year_total#74] -Arguments: hashpartitioning(customer_id#73, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [2]: [customer_id#74, year_total#75] +Arguments: hashpartitioning(customer_id#74, 5), ENSURE_REQUIREMENTS, [id=#76] (77) Sort [codegen id : 34] -Input [2]: [customer_id#73, year_total#74] -Arguments: [customer_id#73 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#74, year_total#75] +Arguments: [customer_id#74 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#73] -Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#38) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#74] +Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#75) / promote_precision(year_total#57)), DecimalType(37,20), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#59) (79) Project [codegen id : 35] Output [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] -Input [9]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57, customer_id#73, year_total#74] +Input [8]: [customer_id#17, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#57, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#59, customer_id#74, year_total#75] (80) TakeOrderedAndProject Input [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] @@ -468,7 +468,7 @@ Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2001)) AND d_year#6 IN (2001, (84) BroadcastExchange Input [2]: [d_date_sk#5, d_year#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (88) @@ -493,10 +493,10 @@ Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (20 (88) BroadcastExchange Input [2]: [d_date_sk#24, d_year#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#23 +Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#23 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt index 99e72fe265b6b..e4c930b0c6f25 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (35) Project [customer_id,customer_first_name,customer_last_name] - SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END] InputAdapter WholeStageCodegen (26) - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] + Project [customer_id,customer_id,customer_first_name,customer_last_name,year_total,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt index 1554259f337c1..a219c72cfe0d4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt @@ -320,93 +320,93 @@ Right keys [1]: [customer_id#54] Join condition: None (53) Project [codegen id : 16] -Output [7]: [customer_id#16, year_total#17, customer_id#33, customer_first_name#34, customer_last_name#35, year_total#36, year_total#55] +Output [6]: [customer_id#16, customer_id#33, customer_first_name#34, customer_last_name#35, year_total#55, CASE WHEN (year_total#17 > 0.00) THEN CheckOverflow((promote_precision(year_total#36) / promote_precision(year_total#17)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#57] Input [8]: [customer_id#16, year_total#17, customer_id#33, customer_first_name#34, customer_last_name#35, year_total#36, customer_id#54, year_total#55] (54) Scan parquet default.customer -Output [4]: [c_customer_sk#57, c_customer_id#58, c_first_name#59, c_last_name#60] +Output [4]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (55) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#57, c_customer_id#58, c_first_name#59, c_last_name#60] +Input [4]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61] (56) Filter [codegen id : 14] -Input [4]: [c_customer_sk#57, c_customer_id#58, c_first_name#59, c_last_name#60] -Condition : (isnotnull(c_customer_sk#57) AND isnotnull(c_customer_id#58)) +Input [4]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61] +Condition : (isnotnull(c_customer_sk#58) AND isnotnull(c_customer_id#59)) (57) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Output [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#25)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] (59) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#61) +Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] +Condition : isnotnull(ws_bill_customer_sk#62) (60) BroadcastExchange -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] +Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] (61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#57] -Right keys [1]: [ws_bill_customer_sk#61] +Left keys [1]: [c_customer_sk#58] +Right keys [1]: [ws_bill_customer_sk#62] Join condition: None (62) Project [codegen id : 14] -Output [5]: [c_customer_id#58, c_first_name#59, c_last_name#60, ws_net_paid#62, ws_sold_date_sk#63] -Input [7]: [c_customer_sk#57, c_customer_id#58, c_first_name#59, c_last_name#60, ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Output [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, ws_sold_date_sk#64] +Input [7]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] (63) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#65, d_year#66] +Output [2]: [d_date_sk#66, d_year#67] (64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#65] +Left keys [1]: [ws_sold_date_sk#64] +Right keys [1]: [d_date_sk#66] Join condition: None (65) Project [codegen id : 14] -Output [5]: [c_customer_id#58, c_first_name#59, c_last_name#60, ws_net_paid#62, d_year#66] -Input [7]: [c_customer_id#58, c_first_name#59, c_last_name#60, ws_net_paid#62, ws_sold_date_sk#63, d_date_sk#65, d_year#66] +Output [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, d_year#67] +Input [7]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, ws_sold_date_sk#64, d_date_sk#66, d_year#67] (66) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#58, c_first_name#59, c_last_name#60, ws_net_paid#62, d_year#66] -Keys [4]: [c_customer_id#58, c_first_name#59, c_last_name#60, d_year#66] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#62))] -Aggregate Attributes [1]: [sum#67] -Results [5]: [c_customer_id#58, c_first_name#59, c_last_name#60, d_year#66, sum#68] +Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, d_year#67] +Keys [4]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#67] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#63))] +Aggregate Attributes [1]: [sum#68] +Results [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#67, sum#69] (67) Exchange -Input [5]: [c_customer_id#58, c_first_name#59, c_last_name#60, d_year#66, sum#68] -Arguments: hashpartitioning(c_customer_id#58, c_first_name#59, c_last_name#60, d_year#66, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#67, sum#69] +Arguments: hashpartitioning(c_customer_id#59, c_first_name#60, c_last_name#61, d_year#67, 5), ENSURE_REQUIREMENTS, [id=#70] (68) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#58, c_first_name#59, c_last_name#60, d_year#66, sum#68] -Keys [4]: [c_customer_id#58, c_first_name#59, c_last_name#60, d_year#66] -Functions [1]: [sum(UnscaledValue(ws_net_paid#62))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#62))#70] -Results [2]: [c_customer_id#58 AS customer_id#71, MakeDecimal(sum(UnscaledValue(ws_net_paid#62))#70,17,2) AS year_total#72] +Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#67, sum#69] +Keys [4]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#67] +Functions [1]: [sum(UnscaledValue(ws_net_paid#63))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#63))#71] +Results [2]: [c_customer_id#59 AS customer_id#72, MakeDecimal(sum(UnscaledValue(ws_net_paid#63))#71,17,2) AS year_total#73] (69) BroadcastExchange -Input [2]: [customer_id#71, year_total#72] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#73] +Input [2]: [customer_id#72, year_total#73] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#74] (70) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#71] -Join condition: (CASE WHEN (year_total#55 > 0.00) THEN CheckOverflow((promote_precision(year_total#72) / promote_precision(year_total#55)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#17 > 0.00) THEN CheckOverflow((promote_precision(year_total#36) / promote_precision(year_total#17)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#72] +Join condition: (CASE WHEN (year_total#55 > 0.00) THEN CheckOverflow((promote_precision(year_total#73) / promote_precision(year_total#55)), DecimalType(37,20), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#57) (71) Project [codegen id : 16] Output [3]: [customer_id#33, customer_first_name#34, customer_last_name#35] -Input [9]: [customer_id#16, year_total#17, customer_id#33, customer_first_name#34, customer_last_name#35, year_total#36, year_total#55, customer_id#71, year_total#72] +Input [8]: [customer_id#16, customer_id#33, customer_first_name#34, customer_last_name#35, year_total#55, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#57, customer_id#72, year_total#73] (72) TakeOrderedAndProject Input [3]: [customer_id#33, customer_first_name#34, customer_last_name#35] @@ -437,7 +437,7 @@ Condition : (((isnotnull(d_year#11) AND (d_year#11 = 2001)) AND d_year#11 IN (20 (76) BroadcastExchange Input [2]: [d_date_sk#10, d_year#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (80) @@ -462,10 +462,10 @@ Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (20 (80) BroadcastExchange Input [2]: [d_date_sk#27, d_year#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#25 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#25 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index ef4d5a103e0d6..4d6a4b6355ee3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -1,8 +1,8 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (16) Project [customer_id,customer_first_name,customer_last_name] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END] + Project [customer_id,customer_id,customer_first_name,customer_last_name,year_total,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] From 02f498dabad1267262bd5317015b1e20a07893f1 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 28 Jul 2021 21:44:58 +0800 Subject: [PATCH 02/17] Fix test --- .../PushDownJoinConditionEvaluation.scala | 28 +- .../q59.sf100/explain.txt | 12 +- .../q59.sf100/simplified.txt | 4 +- .../approved-plans-modified/q59/explain.txt | 12 +- .../q59/simplified.txt | 4 +- .../q65.sf100/explain.txt | 297 ++++++----- .../q65.sf100/simplified.txt | 49 +- .../approved-plans-modified/q65/explain.txt | 94 ++-- .../q65/simplified.txt | 4 +- .../approved-plans-v1_4/q1.sf100/explain.txt | 313 ++++++------ .../q1.sf100/simplified.txt | 47 +- .../approved-plans-v1_4/q1/explain.txt | 295 ++++++----- .../approved-plans-v1_4/q1/simplified.txt | 47 +- .../approved-plans-v1_4/q11/explain.txt | 82 +-- .../approved-plans-v1_4/q11/simplified.txt | 4 +- .../approved-plans-v1_4/q15/explain.txt | 141 +++--- .../approved-plans-v1_4/q15/simplified.txt | 11 +- .../approved-plans-v1_4/q2.sf100/explain.txt | 12 +- .../q2.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q2/explain.txt | 12 +- .../approved-plans-v1_4/q2/simplified.txt | 4 +- .../q24a.sf100/explain.txt | 242 ++++----- .../q24a.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q24a/explain.txt | 362 ++++++------- .../approved-plans-v1_4/q24a/simplified.txt | 21 +- .../q24b.sf100/explain.txt | 242 ++++----- .../q24b.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q24b/explain.txt | 362 ++++++------- .../approved-plans-v1_4/q24b/simplified.txt | 21 +- .../approved-plans-v1_4/q30.sf100/explain.txt | 110 ++-- .../q30.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q30/explain.txt | 333 ++++++------ .../approved-plans-v1_4/q30/simplified.txt | 61 ++- .../approved-plans-v1_4/q31.sf100/explain.txt | 254 +++++----- .../q31.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q31/explain.txt | 473 +++++++++-------- .../approved-plans-v1_4/q31/simplified.txt | 175 ++++--- .../approved-plans-v1_4/q32.sf100/explain.txt | 83 ++- .../q32.sf100/simplified.txt | 13 +- .../approved-plans-v1_4/q32/explain.txt | 88 ++-- .../approved-plans-v1_4/q32/simplified.txt | 4 +- .../approved-plans-v1_4/q4/explain.txt | 244 ++++----- .../approved-plans-v1_4/q4/simplified.txt | 6 +- .../approved-plans-v1_4/q47/explain.txt | 40 +- .../approved-plans-v1_4/q47/simplified.txt | 4 +- .../approved-plans-v1_4/q57/explain.txt | 40 +- .../approved-plans-v1_4/q57/simplified.txt | 4 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 400 +++++++-------- .../q58.sf100/simplified.txt | 182 ++++--- .../approved-plans-v1_4/q58/explain.txt | 395 +++++++-------- .../approved-plans-v1_4/q58/simplified.txt | 167 +++--- .../approved-plans-v1_4/q59.sf100/explain.txt | 12 +- .../q59.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q59/explain.txt | 12 +- .../approved-plans-v1_4/q59/simplified.txt | 4 +- .../approved-plans-v1_4/q65.sf100/explain.txt | 321 ++++++------ .../q65.sf100/simplified.txt | 49 +- .../approved-plans-v1_4/q65/explain.txt | 94 ++-- .../approved-plans-v1_4/q65/simplified.txt | 4 +- .../q72.sf100/simplified.txt | 44 +- .../approved-plans-v1_4/q72/simplified.txt | 4 +- .../approved-plans-v1_4/q8.sf100/explain.txt | 303 ++++++----- .../q8.sf100/simplified.txt | 11 +- .../approved-plans-v1_4/q8/explain.txt | 283 +++++------ .../approved-plans-v1_4/q8/simplified.txt | 15 +- .../approved-plans-v1_4/q81.sf100/explain.txt | 110 ++-- .../q81.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q81/explain.txt | 325 ++++++------ .../approved-plans-v1_4/q81/simplified.txt | 61 ++- .../approved-plans-v1_4/q92.sf100/explain.txt | 101 ++-- .../q92.sf100/simplified.txt | 13 +- .../approved-plans-v1_4/q92/explain.txt | 98 ++-- .../approved-plans-v1_4/q92/simplified.txt | 2 +- .../approved-plans-v2_7/q11/explain.txt | 82 +-- .../approved-plans-v2_7/q11/simplified.txt | 4 +- .../approved-plans-v2_7/q24.sf100/explain.txt | 476 ++++++++---------- .../q24.sf100/simplified.txt | 24 +- .../approved-plans-v2_7/q24/explain.txt | 376 +++++++------- .../approved-plans-v2_7/q24/simplified.txt | 21 +- .../approved-plans-v2_7/q47/explain.txt | 40 +- .../approved-plans-v2_7/q47/simplified.txt | 4 +- .../approved-plans-v2_7/q57/explain.txt | 40 +- .../approved-plans-v2_7/q57/simplified.txt | 4 +- .../q72.sf100/simplified.txt | 44 +- .../approved-plans-v2_7/q72/simplified.txt | 4 +- .../approved-plans-v2_7/q74/explain.txt | 82 +-- .../approved-plans-v2_7/q74/simplified.txt | 4 +- 87 files changed, 4295 insertions(+), 4592 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluation.scala index f439e02c86b81..60f6e86eab347 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluation.scala @@ -25,10 +25,12 @@ import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN /** * Push down join condition evaluation to reduce eval expressions in join condition. */ -object PushDownJoinConditionEvaluation extends Rule[LogicalPlan] with PredicateHelper { +object PushDownJoinConditionEvaluation extends Rule[LogicalPlan] + with JoinSelectionHelper with PredicateHelper { + def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning( _.containsPattern(JOIN), ruleId) { - case j @ Join(left, right, _, Some(condition), _) => + case j @ Join(left, right, _, Some(condition), _) if !canPlanAsBroadcastHashJoin(j, conf) => val expressions = splitConjunctivePredicates(condition).flatMap(_.children).flatMap { case e: Expression if e.children.nonEmpty => Seq(e) case _ => Nil @@ -40,18 +42,22 @@ object PushDownJoinConditionEvaluation extends Rule[LogicalPlan] with PredicateH val leftAlias = leftKeys.map(e => Alias(e, e.sql)()) val rightAlias = rightKeys.map(e => Alias(e, e.sql)()) - val newLeft = if (leftAlias.nonEmpty) Project(left.output ++ leftAlias, left) else left - val newRight = if (rightAlias.nonEmpty) Project(right.output ++ rightAlias, right) else right - - val map = leftKeys.zip(leftAlias).toMap ++ rightKeys.zip(rightAlias).toMap - val newCondition = if (leftAlias.nonEmpty || rightAlias.nonEmpty) { - condition.transformDown { - case e: Expression if e.references.nonEmpty && map.contains(e) => map(e).toAttribute + if (leftAlias.nonEmpty || rightAlias.nonEmpty) { + val pushedPairs = leftKeys.zip(leftAlias).toMap ++ rightKeys.zip(rightAlias).toMap + val newLeft = Project(left.output ++ leftAlias, left) + val newRight = Project(right.output ++ rightAlias, right) + val newCondition = if (leftAlias.nonEmpty || rightAlias.nonEmpty) { + condition.transformDown { + case e: Expression if e.references.nonEmpty && pushedPairs.contains(e) => + pushedPairs(e).toAttribute + } + } else { + condition } + Project(j.output, j.copy(left = newLeft, right = newRight, condition = Some(newCondition))) } else { - condition + j } - j.copy(left = newLeft, right = newRight, condition = Some(newCondition)) } } diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt index da204343a0d87..d20f7cf4a2857 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt @@ -270,21 +270,21 @@ Right keys [1]: [d_week_seq#83] Join condition: None (47) Project [codegen id : 9] -Output [8]: [s_store_id#80 AS s_store_id2#85, sun_sales#30 AS sun_sales2#86, mon_sales#31 AS mon_sales2#87, wed_sales#33 AS wed_sales2#88, thu_sales#34 AS thu_sales2#89, fri_sales#35 AS fri_sales2#90, sat_sales#36 AS sat_sales2#91, (d_week_seq#58 - 52) AS (x.d_week_seq2 - 52)#92] +Output [8]: [d_week_seq#58 AS d_week_seq2#85, s_store_id#80 AS s_store_id2#86, sun_sales#30 AS sun_sales2#87, mon_sales#31 AS mon_sales2#88, wed_sales#33 AS wed_sales2#89, thu_sales#34 AS thu_sales2#90, fri_sales#35 AS fri_sales2#91, sat_sales#36 AS sat_sales2#92] Input [9]: [d_week_seq#58, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#80, d_week_seq#83] (48) BroadcastExchange -Input [8]: [s_store_id2#85, sun_sales2#86, mon_sales2#87, wed_sales2#88, thu_sales2#89, fri_sales2#90, sat_sales2#91, (x.d_week_seq2 - 52)#92] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[7, int, true]),false), [id=#93] +Input [8]: [d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, wed_sales2#89, thu_sales2#90, fri_sales2#91, sat_sales2#92] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#93] (49) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#85, (x.d_week_seq2 - 52)#92] +Right keys [2]: [s_store_id2#86, (d_week_seq2#85 - 52)] Join condition: None (50) Project [codegen id : 10] -Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#86)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#94, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#87)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#95, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#96, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#88)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#97, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#89)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#98, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#90)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#99, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#91)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#100] -Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, s_store_id2#85, sun_sales2#86, mon_sales2#87, wed_sales2#88, thu_sales2#89, fri_sales2#90, sat_sales2#91, (x.d_week_seq2 - 52)#92] +Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#87)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#94, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#88)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#95, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#96, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#89)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#97, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#90)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#98, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#91)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#99, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#92)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#100] +Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, wed_sales2#89, thu_sales2#90, fri_sales2#91, sat_sales2#92] (51) TakeOrderedAndProject Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#94, (mon_sales1 / mon_sales2)#95, (tue_sales1 / tue_sales1)#96, (wed_sales1 / wed_sales2)#97, (thu_sales1 / thu_sales2)#98, (fri_sales1 / fri_sales2)#99, (sat_sales1 / sat_sales2)#100] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt index 604373b3caf7c..eae7970dd89a0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales1),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] WholeStageCodegen (10) Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,(x.d_week_seq2 - 52)] + BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] @@ -42,7 +42,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Project [s_store_id,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + Project [d_week_seq,s_store_id,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt index da204343a0d87..d20f7cf4a2857 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt @@ -270,21 +270,21 @@ Right keys [1]: [d_week_seq#83] Join condition: None (47) Project [codegen id : 9] -Output [8]: [s_store_id#80 AS s_store_id2#85, sun_sales#30 AS sun_sales2#86, mon_sales#31 AS mon_sales2#87, wed_sales#33 AS wed_sales2#88, thu_sales#34 AS thu_sales2#89, fri_sales#35 AS fri_sales2#90, sat_sales#36 AS sat_sales2#91, (d_week_seq#58 - 52) AS (x.d_week_seq2 - 52)#92] +Output [8]: [d_week_seq#58 AS d_week_seq2#85, s_store_id#80 AS s_store_id2#86, sun_sales#30 AS sun_sales2#87, mon_sales#31 AS mon_sales2#88, wed_sales#33 AS wed_sales2#89, thu_sales#34 AS thu_sales2#90, fri_sales#35 AS fri_sales2#91, sat_sales#36 AS sat_sales2#92] Input [9]: [d_week_seq#58, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#80, d_week_seq#83] (48) BroadcastExchange -Input [8]: [s_store_id2#85, sun_sales2#86, mon_sales2#87, wed_sales2#88, thu_sales2#89, fri_sales2#90, sat_sales2#91, (x.d_week_seq2 - 52)#92] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[7, int, true]),false), [id=#93] +Input [8]: [d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, wed_sales2#89, thu_sales2#90, fri_sales2#91, sat_sales2#92] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#93] (49) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#85, (x.d_week_seq2 - 52)#92] +Right keys [2]: [s_store_id2#86, (d_week_seq2#85 - 52)] Join condition: None (50) Project [codegen id : 10] -Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#86)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#94, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#87)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#95, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#96, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#88)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#97, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#89)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#98, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#90)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#99, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#91)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#100] -Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, s_store_id2#85, sun_sales2#86, mon_sales2#87, wed_sales2#88, thu_sales2#89, fri_sales2#90, sat_sales2#91, (x.d_week_seq2 - 52)#92] +Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#87)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#94, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#88)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#95, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#96, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#89)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#97, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#90)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#98, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#91)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#99, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#92)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#100] +Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, wed_sales2#89, thu_sales2#90, fri_sales2#91, sat_sales2#92] (51) TakeOrderedAndProject Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#94, (mon_sales1 / mon_sales2)#95, (tue_sales1 / tue_sales1)#96, (wed_sales1 / wed_sales2)#97, (thu_sales1 / thu_sales2)#98, (fri_sales1 / fri_sales2)#99, (sat_sales1 / sat_sales2)#100] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/simplified.txt index 604373b3caf7c..eae7970dd89a0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales1),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] WholeStageCodegen (10) Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,(x.d_week_seq2 - 52)] + BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] @@ -42,7 +42,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Project [s_store_id,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + Project [d_week_seq,s_store_id,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt index 2de610eed5e3b..7066bd1ed142e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt @@ -1,43 +1,42 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Project (38) - +- * BroadcastHashJoin Inner BuildLeft (37) - :- BroadcastExchange (33) - : +- * Project (32) - : +- * BroadcastHashJoin Inner BuildLeft (31) - : :- BroadcastExchange (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (11) - : : : +- * Filter (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (24) - : : +- * HashAggregate (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- * HashAggregate (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet default.store_sales (12) - : : +- ReusedExchange (15) - : +- * Filter (30) - : +- * ColumnarToRow (29) - : +- Scan parquet default.store (28) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.item (34) +TakeOrderedAndProject (38) ++- * Project (37) + +- * BroadcastHashJoin Inner BuildLeft (36) + :- BroadcastExchange (32) + : +- * Project (31) + : +- * BroadcastHashJoin Inner BuildLeft (30) + : :- BroadcastExchange (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Filter (10) + : : : +- * HashAggregate (9) + : : : +- Exchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (23) + : : +- * HashAggregate (22) + : : +- Exchange (21) + : : +- * HashAggregate (20) + : : +- * HashAggregate (19) + : : +- Exchange (18) + : : +- * HashAggregate (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Filter (13) + : : : +- * ColumnarToRow (12) + : : : +- Scan parquet default.store_sales (11) + : : +- ReusedExchange (14) + : +- * Filter (29) + : +- * ColumnarToRow (28) + : +- Scan parquet default.store (27) + +- * Filter (35) + +- * ColumnarToRow (34) + +- Scan parquet default.item (33) (1) Scan parquet default.store_sales @@ -55,7 +54,7 @@ Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4] Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) -(4) ReusedExchange [Reuses operator id: 44] +(4) ReusedExchange [Reuses operator id: 43] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 2] @@ -89,176 +88,172 @@ Results [3]: [ss_store_sk#2, ss_item_sk#1, MakeDecimal(sum(UnscaledValue(ss_sale Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] Condition : isnotnull(revenue#11) -(11) Project [codegen id : 7] -Output [4]: [ss_store_sk#2, ss_item_sk#1, revenue#11, cast(revenue#11 as decimal(23,7)) AS CAST(sc.revenue AS DECIMAL(23,7))#12] -Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] - -(12) Scan parquet default.store_sales -Output [4]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16] +(11) Scan parquet default.store_sales +Output [4]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#16), (ss_sold_date_sk#16 >= 2451911), (ss_sold_date_sk#16 <= 2452275), dynamicpruningexpression(ss_sold_date_sk#16 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ss_sold_date_sk#15), (ss_sold_date_sk#15 >= 2451911), (ss_sold_date_sk#15 <= 2452275), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 4] -Input [4]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16] +(12) ColumnarToRow [codegen id : 4] +Input [4]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -(14) Filter [codegen id : 4] -Input [4]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16] -Condition : isnotnull(ss_store_sk#14) +(13) Filter [codegen id : 4] +Input [4]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +Condition : isnotnull(ss_store_sk#13) -(15) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#17] +(14) ReusedExchange [Reuses operator id: 43] +Output [1]: [d_date_sk#16] -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#16] -Right keys [1]: [d_date_sk#17] +(15) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#15] +Right keys [1]: [d_date_sk#16] Join condition: None -(17) Project [codegen id : 4] -Output [3]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15] -Input [5]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16, d_date_sk#17] +(16) Project [codegen id : 4] +Output [3]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14] +Input [5]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#16] -(18) HashAggregate [codegen id : 4] -Input [3]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15] -Keys [2]: [ss_store_sk#14, ss_item_sk#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#15))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [ss_store_sk#14, ss_item_sk#13, sum#19] +(17) HashAggregate [codegen id : 4] +Input [3]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14] +Keys [2]: [ss_store_sk#13, ss_item_sk#12] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] +Aggregate Attributes [1]: [sum#17] +Results [3]: [ss_store_sk#13, ss_item_sk#12, sum#18] -(19) Exchange -Input [3]: [ss_store_sk#14, ss_item_sk#13, sum#19] -Arguments: hashpartitioning(ss_store_sk#14, ss_item_sk#13, 5), ENSURE_REQUIREMENTS, [id=#20] +(18) Exchange +Input [3]: [ss_store_sk#13, ss_item_sk#12, sum#18] +Arguments: hashpartitioning(ss_store_sk#13, ss_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#19] -(20) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#14, ss_item_sk#13, sum#19] -Keys [2]: [ss_store_sk#14, ss_item_sk#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#15))#21] -Results [2]: [ss_store_sk#14, MakeDecimal(sum(UnscaledValue(ss_sales_price#15))#21,17,2) AS revenue#22] - -(21) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#14, revenue#22] -Keys [1]: [ss_store_sk#14] -Functions [1]: [partial_avg(revenue#22)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ss_store_sk#14, sum#25, count#26] - -(22) Exchange -Input [3]: [ss_store_sk#14, sum#25, count#26] -Arguments: hashpartitioning(ss_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#27] - -(23) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#14, sum#25, count#26] -Keys [1]: [ss_store_sk#14] -Functions [1]: [avg(revenue#22)] -Aggregate Attributes [1]: [avg(revenue#22)#28] -Results [2]: [ss_store_sk#14, CheckOverflow((0.100000 * promote_precision(avg(revenue#22)#28)), DecimalType(23,7), true) AS (0.100000BD * sb.ave)#29] - -(24) BroadcastExchange -Input [2]: [ss_store_sk#14, (0.100000BD * sb.ave)#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +(19) HashAggregate [codegen id : 5] +Input [3]: [ss_store_sk#13, ss_item_sk#12, sum#18] +Keys [2]: [ss_store_sk#13, ss_item_sk#12] +Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] +Results [2]: [ss_store_sk#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS revenue#21] -(25) BroadcastHashJoin [codegen id : 7] +(20) HashAggregate [codegen id : 5] +Input [2]: [ss_store_sk#13, revenue#21] +Keys [1]: [ss_store_sk#13] +Functions [1]: [partial_avg(revenue#21)] +Aggregate Attributes [2]: [sum#22, count#23] +Results [3]: [ss_store_sk#13, sum#24, count#25] + +(21) Exchange +Input [3]: [ss_store_sk#13, sum#24, count#25] +Arguments: hashpartitioning(ss_store_sk#13, 5), ENSURE_REQUIREMENTS, [id=#26] + +(22) HashAggregate [codegen id : 6] +Input [3]: [ss_store_sk#13, sum#24, count#25] +Keys [1]: [ss_store_sk#13] +Functions [1]: [avg(revenue#21)] +Aggregate Attributes [1]: [avg(revenue#21)#27] +Results [2]: [ss_store_sk#13, avg(revenue#21)#27 AS ave#28] + +(23) BroadcastExchange +Input [2]: [ss_store_sk#13, ave#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] + +(24) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [ss_store_sk#14] -Join condition: (CAST(sc.revenue AS DECIMAL(23,7))#12 <= (0.100000BD * sb.ave)#29) +Right keys [1]: [ss_store_sk#13] +Join condition: (cast(revenue#11 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#28)), DecimalType(23,7), true)) -(26) Project [codegen id : 7] +(25) Project [codegen id : 7] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] -Input [6]: [ss_store_sk#2, ss_item_sk#1, revenue#11, CAST(sc.revenue AS DECIMAL(23,7))#12, ss_store_sk#14, (0.100000BD * sb.ave)#29] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#11, ss_store_sk#13, ave#28] -(27) BroadcastExchange +(26) BroadcastExchange Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] -(28) Scan parquet default.store -Output [2]: [s_store_sk#32, s_store_name#33] +(27) Scan parquet default.store +Output [2]: [s_store_sk#31, s_store_name#32] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(29) ColumnarToRow -Input [2]: [s_store_sk#32, s_store_name#33] +(28) ColumnarToRow +Input [2]: [s_store_sk#31, s_store_name#32] -(30) Filter -Input [2]: [s_store_sk#32, s_store_name#33] -Condition : isnotnull(s_store_sk#32) +(29) Filter +Input [2]: [s_store_sk#31, s_store_name#32] +Condition : isnotnull(s_store_sk#31) -(31) BroadcastHashJoin [codegen id : 8] +(30) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#32] +Right keys [1]: [s_store_sk#31] Join condition: None -(32) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#11, s_store_name#33] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#11, s_store_sk#32, s_store_name#33] +(31) Project [codegen id : 8] +Output [3]: [ss_item_sk#1, revenue#11, s_store_name#32] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#11, s_store_sk#31, s_store_name#32] -(33) BroadcastExchange -Input [3]: [ss_item_sk#1, revenue#11, s_store_name#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +(32) BroadcastExchange +Input [3]: [ss_item_sk#1, revenue#11, s_store_name#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] -(34) Scan parquet default.item -Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +(33) Scan parquet default.item +Output [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(35) ColumnarToRow -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +(34) ColumnarToRow +Input [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] -(36) Filter -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Condition : isnotnull(i_item_sk#35) +(35) Filter +Input [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] +Condition : isnotnull(i_item_sk#34) -(37) BroadcastHashJoin [codegen id : 9] +(36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#35] +Right keys [1]: [i_item_sk#34] Join condition: None -(38) Project [codegen id : 9] -Output [6]: [s_store_name#33, i_item_desc#36, revenue#11, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Input [8]: [ss_item_sk#1, revenue#11, s_store_name#33, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +(37) Project [codegen id : 9] +Output [6]: [s_store_name#32, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] +Input [8]: [ss_item_sk#1, revenue#11, s_store_name#32, i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] -(39) TakeOrderedAndProject -Input [6]: [s_store_name#33, i_item_desc#36, revenue#11, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: 100, [s_store_name#33 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#33, i_item_desc#36, revenue#11, i_current_price#37, i_wholesale_cost#38, i_brand#39] +(38) TakeOrderedAndProject +Input [6]: [s_store_name#32, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] +Arguments: 100, [s_store_name#32 ASC NULLS FIRST, i_item_desc#35 ASC NULLS FIRST], [s_store_name#32, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (44) -+- * Project (43) - +- * Filter (42) - +- * ColumnarToRow (41) - +- Scan parquet default.date_dim (40) +BroadcastExchange (43) ++- * Project (42) + +- * Filter (41) + +- * ColumnarToRow (40) + +- Scan parquet default.date_dim (39) -(40) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#40] +(39) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), GreaterThanOrEqual(d_date_sk,2451911), LessThanOrEqual(d_date_sk,2452275), IsNotNull(d_date_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#40] +(40) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#39] -(42) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#40] -Condition : (((((isnotnull(d_month_seq#40) AND (d_month_seq#40 >= 1212)) AND (d_month_seq#40 <= 1223)) AND (d_date_sk#6 >= 2451911)) AND (d_date_sk#6 <= 2452275)) AND isnotnull(d_date_sk#6)) +(41) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#39] +Condition : (((((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1212)) AND (d_month_seq#39 <= 1223)) AND (d_date_sk#6 >= 2451911)) AND (d_date_sk#6 <= 2452275)) AND isnotnull(d_date_sk#6)) -(43) Project [codegen id : 1] +(42) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_month_seq#40] +Input [2]: [d_date_sk#6, d_month_seq#39] -(44) BroadcastExchange +(43) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] -Subquery:2 Hosting operator id = 12 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt index ad18e1e3e80a3..fd2cf0d33f543 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt @@ -11,34 +11,33 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastExchange #2 WholeStageCodegen (7) Project [ss_store_sk,ss_item_sk,revenue] - BroadcastHashJoin [ss_store_sk,ss_store_sk,CAST(sc.revenue AS DECIMAL(23,7)),(0.100000BD * sb.ave)] - Project [ss_store_sk,ss_item_sk,revenue] - Filter [revenue] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - InputAdapter - Exchange [ss_store_sk,ss_item_sk] #3 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 + BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] + Filter [revenue] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] + InputAdapter + Exchange [ss_store_sk,ss_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #5 WholeStageCodegen (6) - HashAggregate [ss_store_sk,sum,count] [avg(revenue),(0.100000BD * sb.ave),sum,count] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] InputAdapter Exchange [ss_store_sk] #6 WholeStageCodegen (5) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt index 5a1c7dab2b6fc..02c9fdd520c10 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt @@ -139,84 +139,84 @@ Right keys [1]: [i_item_sk#15] Join condition: None (22) Project [codegen id : 9] -Output [8]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19, cast(revenue#13 as decimal(23,7)) AS CAST(sc.revenue AS DECIMAL(23,7))#21] +Output [7]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19] Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#13, i_item_sk#15, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19] (23) Scan parquet default.store_sales -Output [4]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Output [4]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), (ss_sold_date_sk#25 >= 2451911), (ss_sold_date_sk#25 <= 2452275), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#24), (ss_sold_date_sk#24 >= 2451911), (ss_sold_date_sk#24 <= 2452275), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [4]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Input [4]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24] (25) Filter [codegen id : 6] -Input [4]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_store_sk#23) +Input [4]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24] +Condition : isnotnull(ss_store_sk#22) (26) ReusedExchange [Reuses operator id: 43] -Output [1]: [d_date_sk#26] +Output [1]: [d_date_sk#25] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#26] +Left keys [1]: [ss_sold_date_sk#24] +Right keys [1]: [d_date_sk#25] Join condition: None (28) Project [codegen id : 6] -Output [3]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24] -Input [5]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#26] +Output [3]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23] +Input [5]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24, d_date_sk#25] (29) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24] -Keys [2]: [ss_store_sk#23, ss_item_sk#22] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] -Aggregate Attributes [1]: [sum#27] -Results [3]: [ss_store_sk#23, ss_item_sk#22, sum#28] +Input [3]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23] +Keys [2]: [ss_store_sk#22, ss_item_sk#21] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#23))] +Aggregate Attributes [1]: [sum#26] +Results [3]: [ss_store_sk#22, ss_item_sk#21, sum#27] (30) Exchange -Input [3]: [ss_store_sk#23, ss_item_sk#22, sum#28] -Arguments: hashpartitioning(ss_store_sk#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ss_store_sk#22, ss_item_sk#21, sum#27] +Arguments: hashpartitioning(ss_store_sk#22, ss_item_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] (31) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#23, ss_item_sk#22, sum#28] -Keys [2]: [ss_store_sk#23, ss_item_sk#22] -Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#24))#30] -Results [2]: [ss_store_sk#23, MakeDecimal(sum(UnscaledValue(ss_sales_price#24))#30,17,2) AS revenue#31] +Input [3]: [ss_store_sk#22, ss_item_sk#21, sum#27] +Keys [2]: [ss_store_sk#22, ss_item_sk#21] +Functions [1]: [sum(UnscaledValue(ss_sales_price#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#23))#29] +Results [2]: [ss_store_sk#22, MakeDecimal(sum(UnscaledValue(ss_sales_price#23))#29,17,2) AS revenue#30] (32) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#23, revenue#31] -Keys [1]: [ss_store_sk#23] -Functions [1]: [partial_avg(revenue#31)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ss_store_sk#23, sum#34, count#35] +Input [2]: [ss_store_sk#22, revenue#30] +Keys [1]: [ss_store_sk#22] +Functions [1]: [partial_avg(revenue#30)] +Aggregate Attributes [2]: [sum#31, count#32] +Results [3]: [ss_store_sk#22, sum#33, count#34] (33) Exchange -Input [3]: [ss_store_sk#23, sum#34, count#35] -Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ss_store_sk#22, sum#33, count#34] +Arguments: hashpartitioning(ss_store_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] (34) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#23, sum#34, count#35] -Keys [1]: [ss_store_sk#23] -Functions [1]: [avg(revenue#31)] -Aggregate Attributes [1]: [avg(revenue#31)#37] -Results [2]: [ss_store_sk#23, CheckOverflow((0.100000 * promote_precision(avg(revenue#31)#37)), DecimalType(23,7), true) AS (0.100000BD * sb.ave)#38] +Input [3]: [ss_store_sk#22, sum#33, count#34] +Keys [1]: [ss_store_sk#22] +Functions [1]: [avg(revenue#30)] +Aggregate Attributes [1]: [avg(revenue#30)#36] +Results [2]: [ss_store_sk#22, avg(revenue#30)#36 AS ave#37] (35) BroadcastExchange -Input [2]: [ss_store_sk#23, (0.100000BD * sb.ave)#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [2]: [ss_store_sk#22, ave#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] -Right keys [1]: [ss_store_sk#23] -Join condition: (CAST(sc.revenue AS DECIMAL(23,7))#21 <= (0.100000BD * sb.ave)#38) +Right keys [1]: [ss_store_sk#22] +Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#37)), DecimalType(23,7), true)) (37) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] -Input [10]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19, CAST(sc.revenue AS DECIMAL(23,7))#21, ss_store_sk#23, (0.100000BD * sb.ave)#38] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19, ss_store_sk#22, ave#37] (38) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] @@ -233,27 +233,27 @@ BroadcastExchange (43) (39) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#40] +Output [2]: [d_date_sk#8, d_month_seq#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), GreaterThanOrEqual(d_date_sk,2451911), LessThanOrEqual(d_date_sk,2452275), IsNotNull(d_date_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#8, d_month_seq#40] +Input [2]: [d_date_sk#8, d_month_seq#39] (41) Filter [codegen id : 1] -Input [2]: [d_date_sk#8, d_month_seq#40] -Condition : (((((isnotnull(d_month_seq#40) AND (d_month_seq#40 >= 1212)) AND (d_month_seq#40 <= 1223)) AND (d_date_sk#8 >= 2451911)) AND (d_date_sk#8 <= 2452275)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#8, d_month_seq#39] +Condition : (((((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1212)) AND (d_month_seq#39 <= 1223)) AND (d_date_sk#8 >= 2451911)) AND (d_date_sk#8 <= 2452275)) AND isnotnull(d_date_sk#8)) (42) Project [codegen id : 1] Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_month_seq#40] +Input [2]: [d_date_sk#8, d_month_seq#39] (43) BroadcastExchange Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt index e407c9cb608b5..4b519f37a58bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] WholeStageCodegen (9) Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_store_sk,ss_store_sk,CAST(sc.revenue AS DECIMAL(23,7)),(0.100000BD * sb.ave)] + BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [s_store_name,ss_store_sk,ss_item_sk,revenue] @@ -45,7 +45,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes InputAdapter BroadcastExchange #5 WholeStageCodegen (8) - HashAggregate [ss_store_sk,sum,count] [avg(revenue),(0.100000BD * sb.ave),sum,count] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] InputAdapter Exchange [ss_store_sk] #6 WholeStageCodegen (7) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt index b2b74d02141cd..f4fcd285bb57c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt @@ -1,48 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- * SortMergeJoin Inner (42) - :- * Sort (36) - : +- Exchange (35) - : +- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (11) - : : : +- * Filter (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_returns (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (25) - : : +- * Filter (24) - : : +- * HashAggregate (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- * HashAggregate (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet default.store_returns (12) - : : +- ReusedExchange (15) - : +- BroadcastExchange (32) - : +- * Project (31) - : +- * Filter (30) - : +- * ColumnarToRow (29) - : +- Scan parquet default.store (28) - +- * Sort (41) - +- Exchange (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet default.customer (37) +TakeOrderedAndProject (43) ++- * Project (42) + +- * SortMergeJoin Inner (41) + :- * Sort (35) + : +- Exchange (34) + : +- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Filter (10) + : : : +- * HashAggregate (9) + : : : +- Exchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_returns (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (24) + : : +- * Filter (23) + : : +- * HashAggregate (22) + : : +- Exchange (21) + : : +- * HashAggregate (20) + : : +- * HashAggregate (19) + : : +- Exchange (18) + : : +- * HashAggregate (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Filter (13) + : : : +- * ColumnarToRow (12) + : : : +- Scan parquet default.store_returns (11) + : : +- ReusedExchange (14) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- * ColumnarToRow (28) + : +- Scan parquet default.store (27) + +- * Sort (40) + +- Exchange (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet default.customer (36) (1) Scan parquet default.store_returns @@ -60,7 +59,7 @@ Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_s Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) -(4) ReusedExchange [Reuses operator id: 49] +(4) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 2] @@ -94,196 +93,192 @@ Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#11, sr_store_sk#2 AS ctr_store Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13] Condition : isnotnull(ctr_total_return#13) -(11) Project [codegen id : 8] -Output [3]: [ctr_customer_sk#11, ctr_store_sk#12, cast(ctr_total_return#13 as decimal(24,7)) AS CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#14] -Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13] - -(12) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18] +(11) Scan parquet default.store_returns +Output [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#17), dynamicpruningexpression(sr_returned_date_sk#17 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18] +(12) ColumnarToRow [codegen id : 4] +Input [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] -(14) Filter [codegen id : 4] -Input [4]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18] -Condition : isnotnull(sr_store_sk#16) +(13) Filter [codegen id : 4] +Input [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] +Condition : isnotnull(sr_store_sk#15) -(15) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#19] +(14) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#18] -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#18] -Right keys [1]: [d_date_sk#19] +(15) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [sr_returned_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None -(17) Project [codegen id : 4] -Output [3]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17] -Input [5]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18, d_date_sk#19] +(16) Project [codegen id : 4] +Output [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] +Input [5]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17, d_date_sk#18] -(18) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17] -Keys [2]: [sr_customer_sk#15, sr_store_sk#16] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#17))] -Aggregate Attributes [1]: [sum#20] -Results [3]: [sr_customer_sk#15, sr_store_sk#16, sum#21] +(17) HashAggregate [codegen id : 4] +Input [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] +Keys [2]: [sr_customer_sk#14, sr_store_sk#15] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#16))] +Aggregate Attributes [1]: [sum#19] +Results [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] -(19) Exchange -Input [3]: [sr_customer_sk#15, sr_store_sk#16, sum#21] -Arguments: hashpartitioning(sr_customer_sk#15, sr_store_sk#16, 5), ENSURE_REQUIREMENTS, [id=#22] +(18) Exchange +Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] +Arguments: hashpartitioning(sr_customer_sk#14, sr_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#21] -(20) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#15, sr_store_sk#16, sum#21] -Keys [2]: [sr_customer_sk#15, sr_store_sk#16] -Functions [1]: [sum(UnscaledValue(sr_return_amt#17))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#17))#23] -Results [2]: [sr_store_sk#16 AS ctr_store_sk#12, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#23,17,2) AS ctr_total_return#13] +(19) HashAggregate [codegen id : 5] +Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] +Keys [2]: [sr_customer_sk#14, sr_store_sk#15] +Functions [1]: [sum(UnscaledValue(sr_return_amt#16))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#16))#22] +Results [2]: [sr_store_sk#15 AS ctr_store_sk#12, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#22,17,2) AS ctr_total_return#13] -(21) HashAggregate [codegen id : 5] +(20) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#12, ctr_total_return#13] Keys [1]: [ctr_store_sk#12] Functions [1]: [partial_avg(ctr_total_return#13)] -Aggregate Attributes [2]: [sum#24, count#25] -Results [3]: [ctr_store_sk#12, sum#26, count#27] +Aggregate Attributes [2]: [sum#23, count#24] +Results [3]: [ctr_store_sk#12, sum#25, count#26] -(22) Exchange -Input [3]: [ctr_store_sk#12, sum#26, count#27] -Arguments: hashpartitioning(ctr_store_sk#12, 5), ENSURE_REQUIREMENTS, [id=#28] +(21) Exchange +Input [3]: [ctr_store_sk#12, sum#25, count#26] +Arguments: hashpartitioning(ctr_store_sk#12, 5), ENSURE_REQUIREMENTS, [id=#27] -(23) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#12, sum#26, count#27] +(22) HashAggregate [codegen id : 6] +Input [3]: [ctr_store_sk#12, sum#25, count#26] Keys [1]: [ctr_store_sk#12] Functions [1]: [avg(ctr_total_return#13)] -Aggregate Attributes [1]: [avg(ctr_total_return#13)#29] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#13)#29) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12 AS ctr_store_sk#12#31] +Aggregate Attributes [1]: [avg(ctr_total_return#13)#28] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#13)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12 AS ctr_store_sk#12#30] -(24) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12#31] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#30) +(23) Filter [codegen id : 6] +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) -(25) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#32] +(24) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#31] -(26) BroadcastHashJoin [codegen id : 8] +(25) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#12] -Right keys [1]: [ctr_store_sk#12#31] -Join condition: (CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#14 > (avg(ctr_total_return) * 1.2)#30) +Right keys [1]: [ctr_store_sk#12#30] +Join condition: (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) -(27) Project [codegen id : 8] +(26) Project [codegen id : 8] Output [2]: [ctr_customer_sk#11, ctr_store_sk#12] -Input [5]: [ctr_customer_sk#11, ctr_store_sk#12, CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#14, (avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12#31] +Input [5]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] -(28) Scan parquet default.store -Output [2]: [s_store_sk#33, s_state#34] +(27) Scan parquet default.store +Output [2]: [s_store_sk#32, s_state#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#33, s_state#34] +(28) ColumnarToRow [codegen id : 7] +Input [2]: [s_store_sk#32, s_state#33] -(30) Filter [codegen id : 7] -Input [2]: [s_store_sk#33, s_state#34] -Condition : ((isnotnull(s_state#34) AND (s_state#34 = TN)) AND isnotnull(s_store_sk#33)) +(29) Filter [codegen id : 7] +Input [2]: [s_store_sk#32, s_state#33] +Condition : ((isnotnull(s_state#33) AND (s_state#33 = TN)) AND isnotnull(s_store_sk#32)) -(31) Project [codegen id : 7] -Output [1]: [s_store_sk#33] -Input [2]: [s_store_sk#33, s_state#34] +(30) Project [codegen id : 7] +Output [1]: [s_store_sk#32] +Input [2]: [s_store_sk#32, s_state#33] -(32) BroadcastExchange -Input [1]: [s_store_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +(31) BroadcastExchange +Input [1]: [s_store_sk#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] -(33) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#12] -Right keys [1]: [s_store_sk#33] +Right keys [1]: [s_store_sk#32] Join condition: None -(34) Project [codegen id : 8] +(33) Project [codegen id : 8] Output [1]: [ctr_customer_sk#11] -Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#33] +Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#32] -(35) Exchange +(34) Exchange Input [1]: [ctr_customer_sk#11] -Arguments: hashpartitioning(ctr_customer_sk#11, 5), ENSURE_REQUIREMENTS, [id=#36] +Arguments: hashpartitioning(ctr_customer_sk#11, 5), ENSURE_REQUIREMENTS, [id=#35] -(36) Sort [codegen id : 9] +(35) Sort [codegen id : 9] Input [1]: [ctr_customer_sk#11] Arguments: [ctr_customer_sk#11 ASC NULLS FIRST], false, 0 -(37) Scan parquet default.customer -Output [2]: [c_customer_sk#37, c_customer_id#38] +(36) Scan parquet default.customer +Output [2]: [c_customer_sk#36, c_customer_id#37] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(38) ColumnarToRow [codegen id : 10] -Input [2]: [c_customer_sk#37, c_customer_id#38] +(37) ColumnarToRow [codegen id : 10] +Input [2]: [c_customer_sk#36, c_customer_id#37] -(39) Filter [codegen id : 10] -Input [2]: [c_customer_sk#37, c_customer_id#38] -Condition : isnotnull(c_customer_sk#37) +(38) Filter [codegen id : 10] +Input [2]: [c_customer_sk#36, c_customer_id#37] +Condition : isnotnull(c_customer_sk#36) -(40) Exchange -Input [2]: [c_customer_sk#37, c_customer_id#38] -Arguments: hashpartitioning(c_customer_sk#37, 5), ENSURE_REQUIREMENTS, [id=#39] +(39) Exchange +Input [2]: [c_customer_sk#36, c_customer_id#37] +Arguments: hashpartitioning(c_customer_sk#36, 5), ENSURE_REQUIREMENTS, [id=#38] -(41) Sort [codegen id : 11] -Input [2]: [c_customer_sk#37, c_customer_id#38] -Arguments: [c_customer_sk#37 ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 11] +Input [2]: [c_customer_sk#36, c_customer_id#37] +Arguments: [c_customer_sk#36 ASC NULLS FIRST], false, 0 -(42) SortMergeJoin [codegen id : 12] +(41) SortMergeJoin [codegen id : 12] Left keys [1]: [ctr_customer_sk#11] -Right keys [1]: [c_customer_sk#37] +Right keys [1]: [c_customer_sk#36] Join condition: None -(43) Project [codegen id : 12] -Output [1]: [c_customer_id#38] -Input [3]: [ctr_customer_sk#11, c_customer_sk#37, c_customer_id#38] +(42) Project [codegen id : 12] +Output [1]: [c_customer_id#37] +Input [3]: [ctr_customer_sk#11, c_customer_sk#36, c_customer_id#37] -(44) TakeOrderedAndProject -Input [1]: [c_customer_id#38] -Arguments: 100, [c_customer_id#38 ASC NULLS FIRST], [c_customer_id#38] +(43) TakeOrderedAndProject +Input [1]: [c_customer_id#37] +Arguments: 100, [c_customer_id#37 ASC NULLS FIRST], [c_customer_id#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (49) -+- * Project (48) - +- * Filter (47) - +- * ColumnarToRow (46) - +- Scan parquet default.date_dim (45) +BroadcastExchange (48) ++- * Project (47) + +- * Filter (46) + +- * ColumnarToRow (45) + +- Scan parquet default.date_dim (44) -(45) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#40] +(44) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_year#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#40] +(45) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#39] -(47) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#40] -Condition : ((isnotnull(d_year#40) AND (d_year#40 = 2000)) AND isnotnull(d_date_sk#6)) +(46) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2000)) AND isnotnull(d_date_sk#6)) -(48) Project [codegen id : 1] +(47) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#40] +Input [2]: [d_date_sk#6, d_year#39] -(49) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] -Subquery:2 Hosting operator id = 12 Hosting Expression = sr_returned_date_sk#18 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#17 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/simplified.txt index 4c1985607f1ad..08e121487a52b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/simplified.txt @@ -11,30 +11,29 @@ TakeOrderedAndProject [c_customer_id] Project [ctr_customer_sk] BroadcastHashJoin [ctr_store_sk,s_store_sk] Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [ctr_store_sk,ctr_store_sk,CAST(ctr1.ctr_total_return AS DECIMAL(24,7)),(avg(ctr_total_return) * 1.2)] - Project [ctr_customer_sk,ctr_store_sk,ctr_total_return] - Filter [ctr_total_return] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - InputAdapter - Exchange [sr_customer_sk,sr_store_sk] #2 - WholeStageCodegen (2) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk,sr_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] + Filter [ctr_total_return] + HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] + InputAdapter + Exchange [sr_customer_sk,sr_store_sk] #2 + WholeStageCodegen (2) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_store_sk,sr_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt index a06086430a253..5066d5c04fa50 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt @@ -1,45 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (11) - : : : +- * Filter (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_returns (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (25) - : : +- * Filter (24) - : : +- * HashAggregate (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- * HashAggregate (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet default.store_returns (12) - : : +- ReusedExchange (15) - : +- BroadcastExchange (32) - : +- * Project (31) - : +- * Filter (30) - : +- * ColumnarToRow (29) - : +- Scan parquet default.store (28) - +- BroadcastExchange (38) - +- * Filter (37) - +- * ColumnarToRow (36) - +- Scan parquet default.customer (35) +TakeOrderedAndProject (40) ++- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Filter (10) + : : : +- * HashAggregate (9) + : : : +- Exchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_returns (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (24) + : : +- * Filter (23) + : : +- * HashAggregate (22) + : : +- Exchange (21) + : : +- * HashAggregate (20) + : : +- * HashAggregate (19) + : : +- Exchange (18) + : : +- * HashAggregate (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Filter (13) + : : : +- * ColumnarToRow (12) + : : : +- Scan parquet default.store_returns (11) + : : +- ReusedExchange (14) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- * ColumnarToRow (28) + : +- Scan parquet default.store (27) + +- BroadcastExchange (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.customer (34) (1) Scan parquet default.store_returns @@ -57,7 +56,7 @@ Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_s Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) -(4) ReusedExchange [Reuses operator id: 46] +(4) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 2] @@ -91,184 +90,180 @@ Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#11, sr_store_sk#2 AS ctr_store Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13] Condition : isnotnull(ctr_total_return#13) -(11) Project [codegen id : 9] -Output [3]: [ctr_customer_sk#11, ctr_store_sk#12, cast(ctr_total_return#13 as decimal(24,7)) AS CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#14] -Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13] - -(12) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18] +(11) Scan parquet default.store_returns +Output [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#17), dynamicpruningexpression(sr_returned_date_sk#17 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18] +(12) ColumnarToRow [codegen id : 4] +Input [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] -(14) Filter [codegen id : 4] -Input [4]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18] -Condition : isnotnull(sr_store_sk#16) +(13) Filter [codegen id : 4] +Input [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] +Condition : isnotnull(sr_store_sk#15) -(15) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#19] +(14) ReusedExchange [Reuses operator id: 45] +Output [1]: [d_date_sk#18] -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#18] -Right keys [1]: [d_date_sk#19] +(15) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [sr_returned_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None -(17) Project [codegen id : 4] -Output [3]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17] -Input [5]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17, sr_returned_date_sk#18, d_date_sk#19] +(16) Project [codegen id : 4] +Output [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] +Input [5]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17, d_date_sk#18] -(18) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#15, sr_store_sk#16, sr_return_amt#17] -Keys [2]: [sr_customer_sk#15, sr_store_sk#16] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#17))] -Aggregate Attributes [1]: [sum#20] -Results [3]: [sr_customer_sk#15, sr_store_sk#16, sum#21] +(17) HashAggregate [codegen id : 4] +Input [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] +Keys [2]: [sr_customer_sk#14, sr_store_sk#15] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#16))] +Aggregate Attributes [1]: [sum#19] +Results [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] -(19) Exchange -Input [3]: [sr_customer_sk#15, sr_store_sk#16, sum#21] -Arguments: hashpartitioning(sr_customer_sk#15, sr_store_sk#16, 5), ENSURE_REQUIREMENTS, [id=#22] +(18) Exchange +Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] +Arguments: hashpartitioning(sr_customer_sk#14, sr_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#21] -(20) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#15, sr_store_sk#16, sum#21] -Keys [2]: [sr_customer_sk#15, sr_store_sk#16] -Functions [1]: [sum(UnscaledValue(sr_return_amt#17))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#17))#23] -Results [2]: [sr_store_sk#16 AS ctr_store_sk#12, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#23,17,2) AS ctr_total_return#13] +(19) HashAggregate [codegen id : 5] +Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] +Keys [2]: [sr_customer_sk#14, sr_store_sk#15] +Functions [1]: [sum(UnscaledValue(sr_return_amt#16))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#16))#22] +Results [2]: [sr_store_sk#15 AS ctr_store_sk#12, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#22,17,2) AS ctr_total_return#13] -(21) HashAggregate [codegen id : 5] +(20) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#12, ctr_total_return#13] Keys [1]: [ctr_store_sk#12] Functions [1]: [partial_avg(ctr_total_return#13)] -Aggregate Attributes [2]: [sum#24, count#25] -Results [3]: [ctr_store_sk#12, sum#26, count#27] +Aggregate Attributes [2]: [sum#23, count#24] +Results [3]: [ctr_store_sk#12, sum#25, count#26] -(22) Exchange -Input [3]: [ctr_store_sk#12, sum#26, count#27] -Arguments: hashpartitioning(ctr_store_sk#12, 5), ENSURE_REQUIREMENTS, [id=#28] +(21) Exchange +Input [3]: [ctr_store_sk#12, sum#25, count#26] +Arguments: hashpartitioning(ctr_store_sk#12, 5), ENSURE_REQUIREMENTS, [id=#27] -(23) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#12, sum#26, count#27] +(22) HashAggregate [codegen id : 6] +Input [3]: [ctr_store_sk#12, sum#25, count#26] Keys [1]: [ctr_store_sk#12] Functions [1]: [avg(ctr_total_return#13)] -Aggregate Attributes [1]: [avg(ctr_total_return#13)#29] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#13)#29) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12 AS ctr_store_sk#12#31] +Aggregate Attributes [1]: [avg(ctr_total_return#13)#28] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#13)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12 AS ctr_store_sk#12#30] -(24) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12#31] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#30) +(23) Filter [codegen id : 6] +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) -(25) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#32] +(24) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#31] -(26) BroadcastHashJoin [codegen id : 9] +(25) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#12] -Right keys [1]: [ctr_store_sk#12#31] -Join condition: (CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#14 > (avg(ctr_total_return) * 1.2)#30) +Right keys [1]: [ctr_store_sk#12#30] +Join condition: (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) -(27) Project [codegen id : 9] +(26) Project [codegen id : 9] Output [2]: [ctr_customer_sk#11, ctr_store_sk#12] -Input [5]: [ctr_customer_sk#11, ctr_store_sk#12, CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#14, (avg(ctr_total_return) * 1.2)#30, ctr_store_sk#12#31] +Input [5]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] -(28) Scan parquet default.store -Output [2]: [s_store_sk#33, s_state#34] +(27) Scan parquet default.store +Output [2]: [s_store_sk#32, s_state#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#33, s_state#34] +(28) ColumnarToRow [codegen id : 7] +Input [2]: [s_store_sk#32, s_state#33] -(30) Filter [codegen id : 7] -Input [2]: [s_store_sk#33, s_state#34] -Condition : ((isnotnull(s_state#34) AND (s_state#34 = TN)) AND isnotnull(s_store_sk#33)) +(29) Filter [codegen id : 7] +Input [2]: [s_store_sk#32, s_state#33] +Condition : ((isnotnull(s_state#33) AND (s_state#33 = TN)) AND isnotnull(s_store_sk#32)) -(31) Project [codegen id : 7] -Output [1]: [s_store_sk#33] -Input [2]: [s_store_sk#33, s_state#34] +(30) Project [codegen id : 7] +Output [1]: [s_store_sk#32] +Input [2]: [s_store_sk#32, s_state#33] -(32) BroadcastExchange -Input [1]: [s_store_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +(31) BroadcastExchange +Input [1]: [s_store_sk#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] -(33) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#12] -Right keys [1]: [s_store_sk#33] +Right keys [1]: [s_store_sk#32] Join condition: None -(34) Project [codegen id : 9] +(33) Project [codegen id : 9] Output [1]: [ctr_customer_sk#11] -Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#33] +Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#32] -(35) Scan parquet default.customer -Output [2]: [c_customer_sk#36, c_customer_id#37] +(34) Scan parquet default.customer +Output [2]: [c_customer_sk#35, c_customer_id#36] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#36, c_customer_id#37] +(35) ColumnarToRow [codegen id : 8] +Input [2]: [c_customer_sk#35, c_customer_id#36] -(37) Filter [codegen id : 8] -Input [2]: [c_customer_sk#36, c_customer_id#37] -Condition : isnotnull(c_customer_sk#36) +(36) Filter [codegen id : 8] +Input [2]: [c_customer_sk#35, c_customer_id#36] +Condition : isnotnull(c_customer_sk#35) -(38) BroadcastExchange -Input [2]: [c_customer_sk#36, c_customer_id#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] +(37) BroadcastExchange +Input [2]: [c_customer_sk#35, c_customer_id#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] -(39) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_customer_sk#11] -Right keys [1]: [c_customer_sk#36] +Right keys [1]: [c_customer_sk#35] Join condition: None -(40) Project [codegen id : 9] -Output [1]: [c_customer_id#37] -Input [3]: [ctr_customer_sk#11, c_customer_sk#36, c_customer_id#37] +(39) Project [codegen id : 9] +Output [1]: [c_customer_id#36] +Input [3]: [ctr_customer_sk#11, c_customer_sk#35, c_customer_id#36] -(41) TakeOrderedAndProject -Input [1]: [c_customer_id#37] -Arguments: 100, [c_customer_id#37 ASC NULLS FIRST], [c_customer_id#37] +(40) TakeOrderedAndProject +Input [1]: [c_customer_id#36] +Arguments: 100, [c_customer_id#36 ASC NULLS FIRST], [c_customer_id#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (46) -+- * Project (45) - +- * Filter (44) - +- * ColumnarToRow (43) - +- Scan parquet default.date_dim (42) +BroadcastExchange (45) ++- * Project (44) + +- * Filter (43) + +- * ColumnarToRow (42) + +- Scan parquet default.date_dim (41) -(42) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#39] +(41) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_year#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#39] +(42) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#38] -(44) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2000)) AND isnotnull(d_date_sk#6)) +(43) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2000)) AND isnotnull(d_date_sk#6)) -(45) Project [codegen id : 1] +(44) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#39] +Input [2]: [d_date_sk#6, d_year#38] -(46) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] -Subquery:2 Hosting operator id = 12 Hosting Expression = sr_returned_date_sk#18 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#17 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt index 362fe4c04804f..f6fdeccf32fdf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -5,30 +5,29 @@ TakeOrderedAndProject [c_customer_id] Project [ctr_customer_sk] BroadcastHashJoin [ctr_store_sk,s_store_sk] Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [ctr_store_sk,ctr_store_sk,CAST(ctr1.ctr_total_return AS DECIMAL(24,7)),(avg(ctr_total_return) * 1.2)] - Project [ctr_customer_sk,ctr_store_sk,ctr_total_return] - Filter [ctr_total_return] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - InputAdapter - Exchange [sr_customer_sk,sr_store_sk] #1 - WholeStageCodegen (2) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk,sr_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] + Filter [ctr_total_return] + HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] + InputAdapter + Exchange [sr_customer_sk,sr_store_sk] #1 + WholeStageCodegen (2) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_store_sk,sr_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt index 264122a9ef09e..3f9a0cb61ea1a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt @@ -325,93 +325,93 @@ Right keys [1]: [customer_id#68] Join condition: None (54) Project [codegen id : 16] -Output [4]: [customer_id#21, customer_preferred_cust_flag#44, year_total#69, CASE WHEN (year_total#22 > 0.00) THEN CheckOverflow((promote_precision(year_total#45) / promote_precision(year_total#22)), DecimalType(38,20), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#71] +Output [5]: [customer_id#21, year_total#22, customer_preferred_cust_flag#44, year_total#45, year_total#69] Input [6]: [customer_id#21, year_total#22, customer_preferred_cust_flag#44, year_total#45, customer_id#68, year_total#69] (55) Scan parquet default.customer -Output [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] +Output [8]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (56) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] +Input [8]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78] (57) Filter [codegen id : 14] -Input [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] -Condition : (isnotnull(c_customer_sk#72) AND isnotnull(c_customer_id#73)) +Input [8]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78] +Condition : (isnotnull(c_customer_sk#71) AND isnotnull(c_customer_id#72)) (58) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] +Output [4]: [ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(ws_sold_date_sk#82), dynamicpruningexpression(ws_sold_date_sk#82 IN dynamicpruning#35)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] +Input [4]: [ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] (60) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] -Condition : isnotnull(ws_bill_customer_sk#80) +Input [4]: [ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] +Condition : isnotnull(ws_bill_customer_sk#79) (61) BroadcastExchange -Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#84] +Input [4]: [ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#83] (62) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#72] -Right keys [1]: [ws_bill_customer_sk#80] +Left keys [1]: [c_customer_sk#71] +Right keys [1]: [ws_bill_customer_sk#79] Join condition: None (63) Project [codegen id : 14] -Output [10]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] -Input [12]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] +Output [10]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] +Input [12]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] (64) ReusedExchange [Reuses operator id: 81] -Output [2]: [d_date_sk#85, d_year#86] +Output [2]: [d_date_sk#84, d_year#85] (65) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#83] -Right keys [1]: [d_date_sk#85] +Left keys [1]: [ws_sold_date_sk#82] +Right keys [1]: [d_date_sk#84] Join condition: None (66) Project [codegen id : 14] -Output [10]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, ws_ext_discount_amt#81, ws_ext_list_price#82, d_year#86] -Input [12]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83, d_date_sk#85, d_year#86] +Output [10]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_ext_discount_amt#80, ws_ext_list_price#81, d_year#85] +Input [12]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82, d_date_sk#84, d_year#85] (67) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, ws_ext_discount_amt#81, ws_ext_list_price#82, d_year#86] -Keys [8]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#86] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#82 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#81 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#87] -Results [9]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#86, sum#88] +Input [10]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_ext_discount_amt#80, ws_ext_list_price#81, d_year#85] +Keys [8]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#80 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#86] +Results [9]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85, sum#87] (68) Exchange -Input [9]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#86, sum#88] -Arguments: hashpartitioning(c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#86, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [9]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85, sum#87] +Arguments: hashpartitioning(c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85, 5), ENSURE_REQUIREMENTS, [id=#88] (69) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#86, sum#88] -Keys [8]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#86] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#82 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#81 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#82 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#81 as decimal(8,2)))), DecimalType(8,2), true)))#90] -Results [2]: [c_customer_id#73 AS customer_id#91, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#82 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#81 as decimal(8,2)))), DecimalType(8,2), true)))#90,18,2) AS year_total#92] +Input [9]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85, sum#87] +Keys [8]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#80 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#80 as decimal(8,2)))), DecimalType(8,2), true)))#89] +Results [2]: [c_customer_id#72 AS customer_id#90, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#80 as decimal(8,2)))), DecimalType(8,2), true)))#89,18,2) AS year_total#91] (70) BroadcastExchange -Input [2]: [customer_id#91, year_total#92] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#93] +Input [2]: [customer_id#90, year_total#91] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#92] (71) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#21] -Right keys [1]: [customer_id#91] -Join condition: (CASE WHEN (year_total#69 > 0.00) THEN CheckOverflow((promote_precision(year_total#92) / promote_precision(year_total#69)), DecimalType(38,20), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#71) +Right keys [1]: [customer_id#90] +Join condition: (CASE WHEN (year_total#69 > 0.00) THEN CheckOverflow((promote_precision(year_total#91) / promote_precision(year_total#69)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#22 > 0.00) THEN CheckOverflow((promote_precision(year_total#45) / promote_precision(year_total#22)), DecimalType(38,20), true) ELSE null END) (72) Project [codegen id : 16] Output [1]: [customer_preferred_cust_flag#44] -Input [6]: [customer_id#21, customer_preferred_cust_flag#44, year_total#69, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#71, customer_id#91, year_total#92] +Input [7]: [customer_id#21, year_total#22, customer_preferred_cust_flag#44, year_total#45, year_total#69, customer_id#90, year_total#91] (73) TakeOrderedAndProject Input [1]: [customer_preferred_cust_flag#44] @@ -442,7 +442,7 @@ Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2001)) AND isnotnull(d_date_ (77) BroadcastExchange Input [2]: [d_date_sk#15, d_year#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#94] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#93] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#34 IN dynamicpruning#35 BroadcastExchange (81) @@ -467,10 +467,10 @@ Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_ (81) BroadcastExchange Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#94] Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#35 +Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#82 IN dynamicpruning#35 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index 394355ed3a4c6..f860a9b8280a8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -1,8 +1,8 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (16) Project [customer_preferred_cust_flag] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END] - Project [customer_id,customer_preferred_cust_flag,year_total,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] Project [customer_id,year_total,customer_preferred_cust_flag,year_total] BroadcastHashJoin [customer_id,customer_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt index dde89e76ce48e..fabc11130f172 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) +TakeOrderedAndProject (22) ++- * HashAggregate (21) + +- Exchange (20) + +- * HashAggregate (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (15) + : +- * BroadcastHashJoin Inner BuildRight (14) : :- * Project (9) : : +- * BroadcastHashJoin Inner BuildRight (8) : : :- * Filter (3) @@ -16,12 +16,11 @@ TakeOrderedAndProject (23) : : +- * Filter (6) : : +- * ColumnarToRow (5) : : +- Scan parquet default.customer (4) - : +- BroadcastExchange (14) - : +- * Project (13) - : +- * Filter (12) - : +- * ColumnarToRow (11) - : +- Scan parquet default.customer_address (10) - +- ReusedExchange (17) + : +- BroadcastExchange (13) + : +- * Filter (12) + : +- * ColumnarToRow (11) + : +- Scan parquet default.customer_address (10) + +- ReusedExchange (16) (1) Scan parquet default.catalog_sales @@ -63,104 +62,100 @@ Right keys [1]: [c_customer_sk#5] Join condition: None (9) Project [codegen id : 4] -Output [4]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, (cs_sales_price#2 > 500.00) AS (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#8] +Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] (10) Scan parquet default.customer_address -Output [3]: [ca_address_sk#9, ca_state#10, ca_zip#11] +Output [3]: [ca_address_sk#8, ca_state#9, ca_zip#10] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (11) ColumnarToRow [codegen id : 2] -Input [3]: [ca_address_sk#9, ca_state#10, ca_zip#11] +Input [3]: [ca_address_sk#8, ca_state#9, ca_zip#10] (12) Filter [codegen id : 2] -Input [3]: [ca_address_sk#9, ca_state#10, ca_zip#11] -Condition : isnotnull(ca_address_sk#9) +Input [3]: [ca_address_sk#8, ca_state#9, ca_zip#10] +Condition : isnotnull(ca_address_sk#8) -(13) Project [codegen id : 2] -Output [3]: [ca_address_sk#9, ca_zip#11, (substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) AS ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#12] -Input [3]: [ca_address_sk#9, ca_state#10, ca_zip#11] +(13) BroadcastExchange +Input [3]: [ca_address_sk#8, ca_state#9, ca_zip#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] -(14) BroadcastExchange -Input [3]: [ca_address_sk#9, ca_zip#11, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] - -(15) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#6] -Right keys [1]: [ca_address_sk#9] -Join condition: (((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#12 OR (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#8) +Right keys [1]: [ca_address_sk#8] +Join condition: ((substr(ca_zip#10, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#9 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) -(16) Project [codegen id : 4] -Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] -Input [7]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#8, ca_address_sk#9, ca_zip#11, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#12] +(15) Project [codegen id : 4] +Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#10] +Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#8, ca_state#9, ca_zip#10] -(17) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#14] +(16) ReusedExchange [Reuses operator id: 27] +Output [1]: [d_date_sk#12] -(18) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#12] Join condition: None -(19) Project [codegen id : 4] -Output [2]: [cs_sales_price#2, ca_zip#11] -Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#14] +(18) Project [codegen id : 4] +Output [2]: [cs_sales_price#2, ca_zip#10] +Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#10, d_date_sk#12] -(20) HashAggregate [codegen id : 4] -Input [2]: [cs_sales_price#2, ca_zip#11] -Keys [1]: [ca_zip#11] +(19) HashAggregate [codegen id : 4] +Input [2]: [cs_sales_price#2, ca_zip#10] +Keys [1]: [ca_zip#10] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [ca_zip#11, sum#16] +Aggregate Attributes [1]: [sum#13] +Results [2]: [ca_zip#10, sum#14] -(21) Exchange -Input [2]: [ca_zip#11, sum#16] -Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, [id=#17] +(20) Exchange +Input [2]: [ca_zip#10, sum#14] +Arguments: hashpartitioning(ca_zip#10, 5), ENSURE_REQUIREMENTS, [id=#15] -(22) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#11, sum#16] -Keys [1]: [ca_zip#11] +(21) HashAggregate [codegen id : 5] +Input [2]: [ca_zip#10, sum#14] +Keys [1]: [ca_zip#10] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#18] -Results [2]: [ca_zip#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#18,17,2) AS sum(cs_sales_price)#19] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#16] +Results [2]: [ca_zip#10, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#16,17,2) AS sum(cs_sales_price)#17] -(23) TakeOrderedAndProject -Input [2]: [ca_zip#11, sum(cs_sales_price)#19] -Arguments: 100, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#19] +(22) TakeOrderedAndProject +Input [2]: [ca_zip#10, sum(cs_sales_price)#17] +Arguments: 100, [ca_zip#10 ASC NULLS FIRST], [ca_zip#10, sum(cs_sales_price)#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * Project (27) - +- * Filter (26) - +- * ColumnarToRow (25) - +- Scan parquet default.date_dim (24) +BroadcastExchange (27) ++- * Project (26) + +- * Filter (25) + +- * ColumnarToRow (24) + +- Scan parquet default.date_dim (23) -(24) Scan parquet default.date_dim -Output [3]: [d_date_sk#14, d_year#20, d_qoy#21] +(23) Scan parquet default.date_dim +Output [3]: [d_date_sk#12, d_year#18, d_qoy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#14, d_year#20, d_qoy#21] +(24) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#12, d_year#18, d_qoy#19] -(26) Filter [codegen id : 1] -Input [3]: [d_date_sk#14, d_year#20, d_qoy#21] -Condition : ((((isnotnull(d_qoy#21) AND isnotnull(d_year#20)) AND (d_qoy#21 = 2)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#14)) +(25) Filter [codegen id : 1] +Input [3]: [d_date_sk#12, d_year#18, d_qoy#19] +Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#12)) -(27) Project [codegen id : 1] -Output [1]: [d_date_sk#14] -Input [3]: [d_date_sk#14, d_year#20, d_qoy#21] +(26) Project [codegen id : 1] +Output [1]: [d_date_sk#12] +Input [3]: [d_date_sk#12, d_year#18, d_qoy#19] -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +(27) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt index b9ea4ccc843a5..0122b48fa2ecb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] Project [cs_sales_price,ca_zip] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sales_price,cs_sold_date_sk,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA'))),(spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] Project [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] Filter [cs_bill_customer_sk] @@ -33,10 +33,9 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - Project [ca_address_sk,ca_zip,ca_state] - Filter [ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip] + Filter [ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt index 3fa91879b4632..c31bb7470648f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt @@ -182,21 +182,21 @@ Right keys [1]: [d_week_seq#69] Join condition: None (32) Project [codegen id : 11] -Output [8]: [sun_sales#35 AS sun_sales2#72, mon_sales#36 AS mon_sales2#73, tue_sales#37 AS tue_sales2#74, wed_sales#38 AS wed_sales2#75, thu_sales#39 AS thu_sales2#76, fri_sales#40 AS fri_sales2#77, sat_sales#41 AS sat_sales2#78, (d_week_seq#53 - 53) AS (z.d_week_seq2 - 53)#79] +Output [8]: [d_week_seq#53 AS d_week_seq2#72, sun_sales#35 AS sun_sales2#73, mon_sales#36 AS mon_sales2#74, tue_sales#37 AS tue_sales2#75, wed_sales#38 AS wed_sales2#76, thu_sales#39 AS thu_sales2#77, fri_sales#40 AS fri_sales2#78, sat_sales#41 AS sat_sales2#79] Input [9]: [d_week_seq#53, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#69] (33) BroadcastExchange -Input [8]: [sun_sales2#72, mon_sales2#73, tue_sales2#74, wed_sales2#75, thu_sales2#76, fri_sales2#77, sat_sales2#78, (z.d_week_seq2 - 53)#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[7, int, true] as bigint)),false), [id=#80] +Input [8]: [d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] +Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#80] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(z.d_week_seq2 - 53)#79] +Right keys [1]: [(d_week_seq2#72 - 53)] Join condition: None (35) Project [codegen id : 12] -Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#72)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#81, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#73)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#82, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#74)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#83, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#75)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#84, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#76)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#85, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#77)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#86, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#78)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#87] -Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, sun_sales2#72, mon_sales2#73, tue_sales2#74, wed_sales2#75, thu_sales2#76, fri_sales2#77, sat_sales2#78, (z.d_week_seq2 - 53)#79] +Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#73)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#81, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#74)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#82, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#75)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#83, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#76)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#84, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#77)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#85, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#78)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#86, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#79)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#87] +Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] (36) Exchange Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#81, round((mon_sales1 / mon_sales2), 2)#82, round((tue_sales1 / tue_sales2), 2)#83, round((wed_sales1 / wed_sales2), 2)#84, round((thu_sales1 / thu_sales2), 2)#85, round((fri_sales1 / fri_sales2), 2)#86, round((sat_sales1 / sat_sales2), 2)#87] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt index 1e81bb28cb047..036f9e8f9caa5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt @@ -4,7 +4,7 @@ WholeStageCodegen (13) Exchange [d_week_seq1] #1 WholeStageCodegen (12) Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [d_week_seq1,(z.d_week_seq2 - 53)] + BroadcastHashJoin [d_week_seq1,d_week_seq2] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] @@ -44,7 +44,7 @@ WholeStageCodegen (13) InputAdapter BroadcastExchange #5 WholeStageCodegen (11) - Project [sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt index 3fa91879b4632..c31bb7470648f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt @@ -182,21 +182,21 @@ Right keys [1]: [d_week_seq#69] Join condition: None (32) Project [codegen id : 11] -Output [8]: [sun_sales#35 AS sun_sales2#72, mon_sales#36 AS mon_sales2#73, tue_sales#37 AS tue_sales2#74, wed_sales#38 AS wed_sales2#75, thu_sales#39 AS thu_sales2#76, fri_sales#40 AS fri_sales2#77, sat_sales#41 AS sat_sales2#78, (d_week_seq#53 - 53) AS (z.d_week_seq2 - 53)#79] +Output [8]: [d_week_seq#53 AS d_week_seq2#72, sun_sales#35 AS sun_sales2#73, mon_sales#36 AS mon_sales2#74, tue_sales#37 AS tue_sales2#75, wed_sales#38 AS wed_sales2#76, thu_sales#39 AS thu_sales2#77, fri_sales#40 AS fri_sales2#78, sat_sales#41 AS sat_sales2#79] Input [9]: [d_week_seq#53, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#69] (33) BroadcastExchange -Input [8]: [sun_sales2#72, mon_sales2#73, tue_sales2#74, wed_sales2#75, thu_sales2#76, fri_sales2#77, sat_sales2#78, (z.d_week_seq2 - 53)#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[7, int, true] as bigint)),false), [id=#80] +Input [8]: [d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] +Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#80] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(z.d_week_seq2 - 53)#79] +Right keys [1]: [(d_week_seq2#72 - 53)] Join condition: None (35) Project [codegen id : 12] -Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#72)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#81, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#73)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#82, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#74)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#83, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#75)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#84, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#76)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#85, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#77)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#86, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#78)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#87] -Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, sun_sales2#72, mon_sales2#73, tue_sales2#74, wed_sales2#75, thu_sales2#76, fri_sales2#77, sat_sales2#78, (z.d_week_seq2 - 53)#79] +Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#73)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#81, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#74)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#82, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#75)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#83, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#76)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#84, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#77)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#85, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#78)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#86, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#79)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#87] +Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] (36) Exchange Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#81, round((mon_sales1 / mon_sales2), 2)#82, round((tue_sales1 / tue_sales2), 2)#83, round((wed_sales1 / wed_sales2), 2)#84, round((thu_sales1 / thu_sales2), 2)#85, round((fri_sales1 / fri_sales2), 2)#86, round((sat_sales1 / sat_sales2), 2)#87] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index 1e81bb28cb047..036f9e8f9caa5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -4,7 +4,7 @@ WholeStageCodegen (13) Exchange [d_week_seq1] #1 WholeStageCodegen (12) Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [d_week_seq1,(z.d_week_seq2 - 53)] + BroadcastHashJoin [d_week_seq1,d_week_seq2] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] @@ -44,7 +44,7 @@ WholeStageCodegen (13) InputAdapter BroadcastExchange #5 WholeStageCodegen (11) - Project [sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt index f1b439b6931cc..a85878eeb26ed 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt @@ -219,65 +219,65 @@ Right keys [1]: [ca_zip#32] Join condition: None (39) Project [codegen id : 11] -Output [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, upper(ca_country#33) AS upper(spark_catalog.default.customer_address.ca_country)#34] +Output [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, ca_country#33] Input [7]: [s_store_sk#25, s_store_name#26, s_state#28, s_zip#29, ca_state#31, ca_zip#32, ca_country#33] (40) BroadcastExchange -Input [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, upper(spark_catalog.default.customer_address.ca_country)#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[4, string, true]),false), [id=#35] +Input [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, ca_country#33] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#34] (41) BroadcastHashJoin [codegen id : 12] Left keys [2]: [ss_store_sk#3, c_birth_country#18] -Right keys [2]: [s_store_sk#25, upper(spark_catalog.default.customer_address.ca_country)#34] +Right keys [2]: [s_store_sk#25, upper(ca_country#33)] Join condition: None (42) Project [codegen id : 12] Output [11]: [ss_net_paid#5, s_store_name#26, s_state#28, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#31] -Input [15]: [ss_store_sk#3, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, upper(spark_catalog.default.customer_address.ca_country)#34] +Input [15]: [ss_store_sk#3, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, ca_country#33] (43) HashAggregate [codegen id : 12] Input [11]: [ss_net_paid#5, s_store_name#26, s_state#28, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#31] Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#36] -Results [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#37] +Aggregate Attributes [1]: [sum#35] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] (44) Exchange -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#37] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#37] (45) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#37] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] (46) HashAggregate [codegen id : 13] -Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, netpaid#40] +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, netpaid#39] Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#26] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#43, isEmpty#44] +Functions [1]: [partial_sum(netpaid#39)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43] (47) Exchange -Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, 5), ENSURE_REQUIREMENTS, [id=#44] (48) HashAggregate [codegen id : 14] -Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#43, isEmpty#44] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43] Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#26] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#46] -Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, sum(netpaid#40)#46 AS paid#47] +Functions [1]: [sum(netpaid#39)] +Aggregate Attributes [1]: [sum(netpaid#39)#45] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, sum(netpaid#39)#45 AS paid#46] (49) Filter [codegen id : 14] -Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#47] -Condition : (isnotnull(paid#47) AND (cast(paid#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#46] +Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#47, [id=#48] * HashAggregate (97) +- Exchange (96) +- * HashAggregate (95) @@ -329,218 +329,218 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (50) Scan parquet default.store_sales -Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] (52) Filter [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +Condition : (((isnotnull(ss_ticket_number#52) AND isnotnull(ss_item_sk#49)) AND isnotnull(ss_store_sk#51)) AND isnotnull(ss_customer_sk#50)) (53) Project [codegen id : 2] -Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] (54) Scan parquet default.store -Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Output [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] (56) Filter [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) +Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] +Condition : (((isnotnull(s_market_id#57) AND (s_market_id#57 = 8)) AND isnotnull(s_store_sk#55)) AND isnotnull(s_zip#59)) (57) Project [codegen id : 1] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Output [4]: [s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] +Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] (58) BroadcastExchange -Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +Input [4]: [s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] (59) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#52] -Right keys [1]: [s_store_sk#56] +Left keys [1]: [ss_store_sk#51] +Right keys [1]: [s_store_sk#55] Join condition: None (60) Project [codegen id : 2] -Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Output [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] +Input [9]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] (61) Exchange -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] +Arguments: hashpartitioning(ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#61] (62) Sort [codegen id : 3] -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] +Arguments: [ss_item_sk#49 ASC NULLS FIRST], false, 0 (63) Scan parquet default.item -Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Output [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] (65) Filter [codegen id : 4] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Condition : isnotnull(i_item_sk#63) +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Condition : isnotnull(i_item_sk#62) (66) Exchange -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: hashpartitioning(i_item_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: hashpartitioning(i_item_sk#62, 5), ENSURE_REQUIREMENTS, [id=#68] (67) Sort [codegen id : 5] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: [i_item_sk#63 ASC NULLS FIRST], false, 0 +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: [i_item_sk#62 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#50] -Right keys [1]: [i_item_sk#63] +Left keys [1]: [ss_item_sk#49] +Right keys [1]: [i_item_sk#62] Join condition: None (69) Project [codegen id : 6] -Output [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Input [13]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Output [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Input [13]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] (70) Exchange -Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: hashpartitioning(ss_customer_sk#50, 5), ENSURE_REQUIREMENTS, [id=#69] (71) Sort [codegen id : 7] -Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: [ss_customer_sk#50 ASC NULLS FIRST], false, 0 (72) ReusedExchange [Reuses operator id: 16] -Output [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Output [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] (73) Sort [codegen id : 9] -Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: [c_customer_sk#70 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#51] -Right keys [1]: [c_customer_sk#71] +Left keys [1]: [ss_customer_sk#50] +Right keys [1]: [c_customer_sk#70] Join condition: None (75) Project [codegen id : 10] -Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Input [16]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Output [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Input [16]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] (76) Exchange -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: hashpartitioning(ss_ticket_number#53, ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: hashpartitioning(ss_ticket_number#52, ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#74] (77) Sort [codegen id : 11] -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: [ss_ticket_number#53 ASC NULLS FIRST, ss_item_sk#50 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST], false, 0 (78) ReusedExchange [Reuses operator id: 26] -Output [2]: [sr_item_sk#76, sr_ticket_number#77] +Output [2]: [sr_item_sk#75, sr_ticket_number#76] (79) Sort [codegen id : 13] -Input [2]: [sr_item_sk#76, sr_ticket_number#77] -Arguments: [sr_ticket_number#77 ASC NULLS FIRST, sr_item_sk#76 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#75, sr_ticket_number#76] +Arguments: [sr_ticket_number#76 ASC NULLS FIRST, sr_item_sk#75 ASC NULLS FIRST], false, 0 (80) SortMergeJoin [codegen id : 14] -Left keys [2]: [ss_ticket_number#53, ss_item_sk#50] -Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] +Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] +Right keys [2]: [sr_ticket_number#76, sr_item_sk#75] Join condition: None (81) Project [codegen id : 14] -Output [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Input [16]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, sr_item_sk#76, sr_ticket_number#77] +Output [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Input [16]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, sr_item_sk#75, sr_ticket_number#76] (82) Exchange -Input [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: hashpartitioning(c_birth_country#74, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: hashpartitioning(c_birth_country#73, s_zip#59, 5), ENSURE_REQUIREMENTS, [id=#77] (83) Sort [codegen id : 15] -Input [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: [c_birth_country#74 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 +Input [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: [c_birth_country#73 ASC NULLS FIRST, s_zip#59 ASC NULLS FIRST], false, 0 (84) Scan parquet default.customer_address -Output [3]: [ca_state#79, ca_zip#80, ca_country#81] +Output [3]: [ca_state#78, ca_zip#79, ca_country#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (85) ColumnarToRow [codegen id : 16] -Input [3]: [ca_state#79, ca_zip#80, ca_country#81] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] (86) Filter [codegen id : 16] -Input [3]: [ca_state#79, ca_zip#80, ca_country#81] -Condition : ((isnotnull(ca_country#81) AND isnotnull(ca_zip#80)) AND isnotnull(upper(ca_country#81))) +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Condition : ((isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) AND isnotnull(upper(ca_country#80))) (87) Project [codegen id : 16] -Output [3]: [ca_state#79, ca_zip#80, upper(ca_country#81) AS upper(spark_catalog.default.customer_address.ca_country)#82] -Input [3]: [ca_state#79, ca_zip#80, ca_country#81] +Output [3]: [ca_state#78, ca_zip#79, upper(ca_country#80) AS upper(spark_catalog.default.customer_address.ca_country)#81] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] (88) Exchange -Input [3]: [ca_state#79, ca_zip#80, upper(spark_catalog.default.customer_address.ca_country)#82] -Arguments: hashpartitioning(upper(spark_catalog.default.customer_address.ca_country)#82, ca_zip#80, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [3]: [ca_state#78, ca_zip#79, upper(spark_catalog.default.customer_address.ca_country)#81] +Arguments: hashpartitioning(upper(spark_catalog.default.customer_address.ca_country)#81, ca_zip#79, 5), ENSURE_REQUIREMENTS, [id=#82] (89) Sort [codegen id : 17] -Input [3]: [ca_state#79, ca_zip#80, upper(spark_catalog.default.customer_address.ca_country)#82] -Arguments: [upper(spark_catalog.default.customer_address.ca_country)#82 ASC NULLS FIRST, ca_zip#80 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#78, ca_zip#79, upper(spark_catalog.default.customer_address.ca_country)#81] +Arguments: [upper(spark_catalog.default.customer_address.ca_country)#81 ASC NULLS FIRST, ca_zip#79 ASC NULLS FIRST], false, 0 (90) SortMergeJoin [codegen id : 18] -Left keys [2]: [c_birth_country#74, s_zip#60] -Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#82, ca_zip#80] +Left keys [2]: [c_birth_country#73, s_zip#59] +Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#81, ca_zip#79] Join condition: None (91) Project [codegen id : 18] -Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#79] -Input [15]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, ca_state#79, ca_zip#80, upper(spark_catalog.default.customer_address.ca_country)#82] +Output [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#78] +Input [15]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, ca_state#78, ca_zip#79, upper(spark_catalog.default.customer_address.ca_country)#81] (92) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#79] -Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum#84] -Results [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#85] +Input [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#78] +Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum#83] +Results [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#84] (93) Exchange -Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#85] -Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#84] +Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#85] (94) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#85] -Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] -Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#87] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#87,17,2) AS netpaid#40] +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#84] +Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] +Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#86] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#86,17,2) AS netpaid#39] (95) HashAggregate [codegen id : 19] -Input [1]: [netpaid#40] +Input [1]: [netpaid#39] Keys: [] -Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#88, count#89] -Results [2]: [sum#90, count#91] +Functions [1]: [partial_avg(netpaid#39)] +Aggregate Attributes [2]: [sum#87, count#88] +Results [2]: [sum#89, count#90] (96) Exchange -Input [2]: [sum#90, count#91] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#92] +Input [2]: [sum#89, count#90] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] (97) HashAggregate [codegen id : 20] -Input [2]: [sum#90, count#91] +Input [2]: [sum#89, count#90] Keys: [] -Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#93] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#93)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#94] +Functions [1]: [avg(netpaid#39)] +Aggregate Attributes [1]: [avg(netpaid#39)#92] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#92)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#93] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt index 5944b47e7a15d..bb65ace10c8e9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt @@ -101,7 +101,7 @@ WholeStageCodegen (14) WholeStageCodegen (12) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [ss_store_sk,c_birth_country,s_store_sk,upper(spark_catalog.default.customer_address.ca_country)] + BroadcastHashJoin [ss_store_sk,c_birth_country,s_store_sk,ca_country] Project [ss_store_sk,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt index 41b4a970960c7..c8d73ab27c82b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) +* Filter (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) :- * Project (33) : +- * BroadcastHashJoin Inner BuildRight (32) : :- * Project (27) @@ -41,11 +41,10 @@ : +- * Filter (30) : +- * ColumnarToRow (29) : +- Scan parquet default.customer (28) - +- BroadcastExchange (38) - +- * Project (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.customer_address (34) + +- BroadcastExchange (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.customer_address (34) (1) Scan parquet default.store_sales @@ -208,238 +207,211 @@ Input [3]: [ca_state#30, ca_zip#31, ca_country#32] Input [3]: [ca_state#30, ca_zip#31, ca_country#32] Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) -(37) Project [codegen id : 8] -Output [3]: [ca_state#30, ca_zip#31, upper(ca_country#32) AS upper(spark_catalog.default.customer_address.ca_country)#33] +(37) BroadcastExchange Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#33] -(38) BroadcastExchange -Input [3]: [ca_state#30, ca_zip#31, upper(spark_catalog.default.customer_address.ca_country)#33] -Arguments: HashedRelationBroadcastMode(List(input[2, string, true], input[1, string, true]),false), [id=#34] - -(39) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 9] Left keys [2]: [c_birth_country#28, s_zip#16] -Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#33, ca_zip#31] +Right keys [2]: [upper(ca_country#32), ca_zip#31] Join condition: None -(40) Project [codegen id : 9] +(39) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, ca_state#30] -Input [15]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, c_birth_country#28, ca_state#30, ca_zip#31, upper(spark_catalog.default.customer_address.ca_country)#33] +Input [15]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, c_birth_country#28, ca_state#30, ca_zip#31, ca_country#32] -(41) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, ca_state#30] Keys [10]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#35] -Results [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#36] +Aggregate Attributes [1]: [sum#34] +Results [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#35] -(42) Exchange -Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#36] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#37] +(41) Exchange +Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#35] +Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#36] -(43) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#36] +(42) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#35] Keys [10]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] -Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#37] +Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#37,17,2) AS netpaid#38] -(44) HashAggregate [codegen id : 10] -Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, netpaid#39] +(43) HashAggregate [codegen id : 10] +Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, netpaid#38] Keys [3]: [c_last_name#27, c_first_name#26, s_store_name#13] -Functions [1]: [partial_sum(netpaid#39)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#42, isEmpty#43] +Functions [1]: [partial_sum(netpaid#38)] +Aggregate Attributes [2]: [sum#39, isEmpty#40] +Results [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42] -(45) Exchange -Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#42, isEmpty#43] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#44] +(44) Exchange +Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42] +Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#43] -(46) HashAggregate [codegen id : 11] -Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#42, isEmpty#43] +(45) HashAggregate [codegen id : 11] +Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42] Keys [3]: [c_last_name#27, c_first_name#26, s_store_name#13] -Functions [1]: [sum(netpaid#39)] -Aggregate Attributes [1]: [sum(netpaid#39)#45] -Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, sum(netpaid#39)#45 AS paid#46] +Functions [1]: [sum(netpaid#38)] +Aggregate Attributes [1]: [sum(netpaid#38)#44] +Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, sum(netpaid#38)#44 AS paid#45] -(47) Filter [codegen id : 11] -Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) +(46) Filter [codegen id : 11] +Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#45] +Condition : (isnotnull(paid#45) AND (cast(paid#45 as decimal(33,8)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (78) -+- Exchange (77) - +- * HashAggregate (76) - +- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (56) - : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : :- * Project (53) - : : : : +- * SortMergeJoin Inner (52) - : : : : :- * Sort (49) - : : : : : +- ReusedExchange (48) - : : : : +- * Sort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- BroadcastExchange (60) - : : +- * Filter (59) - : : +- * ColumnarToRow (58) - : : +- Scan parquet default.item (57) - : +- ReusedExchange (63) - +- BroadcastExchange (70) - +- * Project (69) - +- * Filter (68) - +- * ColumnarToRow (67) - +- Scan parquet default.customer_address (66) - - -(48) ReusedExchange [Reuses operator id: 5] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) Sort [codegen id : 2] -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST], false, 0 - -(50) ReusedExchange [Reuses operator id: 11] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) Sort [codegen id : 4] -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST], false, 0 - -(52) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] -Right keys [2]: [sr_ticket_number#55, sr_item_sk#54] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#46, [id=#47] +* HashAggregate (73) ++- Exchange (72) + +- * HashAggregate (71) + +- * HashAggregate (70) + +- Exchange (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Project (52) + : : : : +- * SortMergeJoin Inner (51) + : : : : :- * Sort (48) + : : : : : +- ReusedExchange (47) + : : : : +- * Sort (50) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (59) + : : +- * Filter (58) + : : +- * ColumnarToRow (57) + : : +- Scan parquet default.item (56) + : +- ReusedExchange (62) + +- ReusedExchange (65) + + +(47) ReusedExchange [Reuses operator id: 5] +Output [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] + +(48) Sort [codegen id : 2] +Input [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] +Arguments: [ss_ticket_number#51 ASC NULLS FIRST, ss_item_sk#48 ASC NULLS FIRST], false, 0 + +(49) ReusedExchange [Reuses operator id: 11] +Output [2]: [sr_item_sk#53, sr_ticket_number#54] + +(50) Sort [codegen id : 4] +Input [2]: [sr_item_sk#53, sr_ticket_number#54] +Arguments: [sr_ticket_number#54 ASC NULLS FIRST, sr_item_sk#53 ASC NULLS FIRST], false, 0 + +(51) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#51, ss_item_sk#48] +Right keys [2]: [sr_ticket_number#54, sr_item_sk#53] Join condition: None -(53) Project [codegen id : 9] -Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] +(52) Project [codegen id : 9] +Output [4]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52] +Input [7]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, sr_item_sk#53, sr_ticket_number#54] -(54) ReusedExchange [Reuses operator id: 19] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] +(53) ReusedExchange [Reuses operator id: 19] +Output [4]: [s_store_sk#55, s_store_name#56, s_state#57, s_zip#58] -(55) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#56] +(54) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#50] +Right keys [1]: [s_store_sk#55] Join condition: None -(56) Project [codegen id : 9] -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] +(55) Project [codegen id : 9] +Output [6]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58] +Input [8]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52, s_store_sk#55, s_store_name#56, s_state#57, s_zip#58] -(57) Scan parquet default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] +(56) Scan parquet default.item +Output [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] +(57) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -(59) Filter [codegen id : 6] -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) +(58) Filter [codegen id : 6] +Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +Condition : isnotnull(i_item_sk#59) -(60) BroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] +(59) BroadcastExchange +Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] -(61) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#49] -Right keys [1]: [i_item_sk#60] +(60) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#48] +Right keys [1]: [i_item_sk#59] Join condition: None -(62) Project [codegen id : 9] -Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] +(61) Project [codegen id : 9] +Output [10]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +Input [12]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -(63) ReusedExchange [Reuses operator id: 31] -Output [4]: [c_customer_sk#67, c_first_name#68, c_last_name#69, c_birth_country#70] +(62) ReusedExchange [Reuses operator id: 31] +Output [4]: [c_customer_sk#66, c_first_name#67, c_last_name#68, c_birth_country#69] -(64) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#50] -Right keys [1]: [c_customer_sk#67] +(63) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#49] +Right keys [1]: [c_customer_sk#66] Join condition: None -(65) Project [codegen id : 9] -Output [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, c_birth_country#70] -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_customer_sk#67, c_first_name#68, c_last_name#69, c_birth_country#70] - -(66) Scan parquet default.customer_address -Output [3]: [ca_state#71, ca_zip#72, ca_country#73] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] -ReadSchema: struct - -(67) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#71, ca_zip#72, ca_country#73] - -(68) Filter [codegen id : 8] -Input [3]: [ca_state#71, ca_zip#72, ca_country#73] -Condition : ((isnotnull(ca_country#73) AND isnotnull(ca_zip#72)) AND isnotnull(upper(ca_country#73))) - -(69) Project [codegen id : 8] -Output [3]: [ca_state#71, ca_zip#72, upper(ca_country#73) AS upper(spark_catalog.default.customer_address.ca_country)#74] -Input [3]: [ca_state#71, ca_zip#72, ca_country#73] +(64) Project [codegen id : 9] +Output [12]: [ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, c_birth_country#69] +Input [14]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_customer_sk#66, c_first_name#67, c_last_name#68, c_birth_country#69] -(70) BroadcastExchange -Input [3]: [ca_state#71, ca_zip#72, upper(spark_catalog.default.customer_address.ca_country)#74] -Arguments: HashedRelationBroadcastMode(List(input[2, string, true], input[1, string, true]),false), [id=#75] +(65) ReusedExchange [Reuses operator id: 37] +Output [3]: [ca_state#70, ca_zip#71, ca_country#72] -(71) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#70, s_zip#59] -Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#74, ca_zip#72] +(66) BroadcastHashJoin [codegen id : 9] +Left keys [2]: [c_birth_country#69, s_zip#58] +Right keys [2]: [upper(ca_country#72), ca_zip#71] Join condition: None -(72) Project [codegen id : 9] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, ca_state#71] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, c_birth_country#70, ca_state#71, ca_zip#72, upper(spark_catalog.default.customer_address.ca_country)#74] - -(73) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, ca_state#71] -Keys [10]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#76] -Results [11]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, sum#77] - -(74) Exchange -Input [11]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, sum#77] -Arguments: hashpartitioning(c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, 5), ENSURE_REQUIREMENTS, [id=#78] - -(75) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, sum#77] -Keys [10]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#79] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#79,17,2) AS netpaid#39] - -(76) HashAggregate [codegen id : 10] -Input [1]: [netpaid#39] +(67) Project [codegen id : 9] +Output [11]: [ss_net_paid#52, s_store_name#56, s_state#57, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, ca_state#70] +Input [15]: [ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, c_birth_country#69, ca_state#70, ca_zip#71, ca_country#72] + +(68) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#52, s_store_name#56, s_state#57, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, ca_state#70] +Keys [10]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [sum#73] +Results [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] + +(69) Exchange +Input [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] +Arguments: hashpartitioning(c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, 5), ENSURE_REQUIREMENTS, [id=#75] + +(70) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] +Keys [10]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61] +Functions [1]: [sum(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#52))#76] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#52))#76,17,2) AS netpaid#38] + +(71) HashAggregate [codegen id : 10] +Input [1]: [netpaid#38] Keys: [] -Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#80, count#81] -Results [2]: [sum#82, count#83] +Functions [1]: [partial_avg(netpaid#38)] +Aggregate Attributes [2]: [sum#77, count#78] +Results [2]: [sum#79, count#80] -(77) Exchange -Input [2]: [sum#82, count#83] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#84] +(72) Exchange +Input [2]: [sum#79, count#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] -(78) HashAggregate [codegen id : 11] -Input [2]: [sum#82, count#83] +(73) HashAggregate [codegen id : 11] +Input [2]: [sum#79, count#80] Keys: [] -Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#85] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#85)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#86] +Functions [1]: [avg(netpaid#38)] +Aggregate Attributes [1]: [avg(netpaid#38)#82] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#82)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#83] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index f60055c66c5cb..2d9ef020540b8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -13,7 +13,7 @@ WholeStageCodegen (11) WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,upper(spark_catalog.default.customer_address.ca_country),ca_zip] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -44,13 +44,7 @@ WholeStageCodegen (11) InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (8) - Project [ca_state,ca_zip,ca_country] - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + ReusedExchange [ca_state,ca_zip,ca_country] #8 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name,s_store_name] #1 @@ -62,7 +56,7 @@ WholeStageCodegen (11) WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,upper(spark_catalog.default.customer_address.ca_country),ca_zip] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -118,8 +112,7 @@ WholeStageCodegen (11) InputAdapter BroadcastExchange #8 WholeStageCodegen (8) - Project [ca_state,ca_zip,ca_country] - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + Filter [ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt index 68170dd60175e..5e87b8e585341 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt @@ -219,65 +219,65 @@ Right keys [1]: [ca_zip#32] Join condition: None (39) Project [codegen id : 11] -Output [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, upper(ca_country#33) AS upper(spark_catalog.default.customer_address.ca_country)#34] +Output [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, ca_country#33] Input [7]: [s_store_sk#25, s_store_name#26, s_state#28, s_zip#29, ca_state#31, ca_zip#32, ca_country#33] (40) BroadcastExchange -Input [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, upper(spark_catalog.default.customer_address.ca_country)#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[4, string, true]),false), [id=#35] +Input [5]: [s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, ca_country#33] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#34] (41) BroadcastHashJoin [codegen id : 12] Left keys [2]: [ss_store_sk#3, c_birth_country#18] -Right keys [2]: [s_store_sk#25, upper(spark_catalog.default.customer_address.ca_country)#34] +Right keys [2]: [s_store_sk#25, upper(ca_country#33)] Join condition: None (42) Project [codegen id : 12] Output [11]: [ss_net_paid#5, s_store_name#26, s_state#28, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#31] -Input [15]: [ss_store_sk#3, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, upper(spark_catalog.default.customer_address.ca_country)#34] +Input [15]: [ss_store_sk#3, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#25, s_store_name#26, s_state#28, ca_state#31, ca_country#33] (43) HashAggregate [codegen id : 12] Input [11]: [ss_net_paid#5, s_store_name#26, s_state#28, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#31] Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#36] -Results [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#37] +Aggregate Attributes [1]: [sum#35] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] (44) Exchange -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#37] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#37] (45) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#37] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] (46) HashAggregate [codegen id : 13] -Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, netpaid#40] +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, netpaid#39] Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#26] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#43, isEmpty#44] +Functions [1]: [partial_sum(netpaid#39)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43] (47) Exchange -Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, 5), ENSURE_REQUIREMENTS, [id=#44] (48) HashAggregate [codegen id : 14] -Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#43, isEmpty#44] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43] Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#26] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#46] -Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, sum(netpaid#40)#46 AS paid#47] +Functions [1]: [sum(netpaid#39)] +Aggregate Attributes [1]: [sum(netpaid#39)#45] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, sum(netpaid#39)#45 AS paid#46] (49) Filter [codegen id : 14] -Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#47] -Condition : (isnotnull(paid#47) AND (cast(paid#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#46] +Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#47, [id=#48] * HashAggregate (97) +- Exchange (96) +- * HashAggregate (95) @@ -329,218 +329,218 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (50) Scan parquet default.store_sales -Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] (52) Filter [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +Condition : (((isnotnull(ss_ticket_number#52) AND isnotnull(ss_item_sk#49)) AND isnotnull(ss_store_sk#51)) AND isnotnull(ss_customer_sk#50)) (53) Project [codegen id : 2] -Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] (54) Scan parquet default.store -Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Output [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] (56) Filter [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) +Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] +Condition : (((isnotnull(s_market_id#57) AND (s_market_id#57 = 8)) AND isnotnull(s_store_sk#55)) AND isnotnull(s_zip#59)) (57) Project [codegen id : 1] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Output [4]: [s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] +Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] (58) BroadcastExchange -Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +Input [4]: [s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] (59) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#52] -Right keys [1]: [s_store_sk#56] +Left keys [1]: [ss_store_sk#51] +Right keys [1]: [s_store_sk#55] Join condition: None (60) Project [codegen id : 2] -Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Output [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] +Input [9]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] (61) Exchange -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] +Arguments: hashpartitioning(ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#61] (62) Sort [codegen id : 3] -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] +Arguments: [ss_item_sk#49 ASC NULLS FIRST], false, 0 (63) Scan parquet default.item -Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Output [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] (65) Filter [codegen id : 4] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Condition : isnotnull(i_item_sk#63) +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Condition : isnotnull(i_item_sk#62) (66) Exchange -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: hashpartitioning(i_item_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: hashpartitioning(i_item_sk#62, 5), ENSURE_REQUIREMENTS, [id=#68] (67) Sort [codegen id : 5] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: [i_item_sk#63 ASC NULLS FIRST], false, 0 +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: [i_item_sk#62 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#50] -Right keys [1]: [i_item_sk#63] +Left keys [1]: [ss_item_sk#49] +Right keys [1]: [i_item_sk#62] Join condition: None (69) Project [codegen id : 6] -Output [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Input [13]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Output [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Input [13]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] (70) Exchange -Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: hashpartitioning(ss_customer_sk#50, 5), ENSURE_REQUIREMENTS, [id=#69] (71) Sort [codegen id : 7] -Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: [ss_customer_sk#50 ASC NULLS FIRST], false, 0 (72) ReusedExchange [Reuses operator id: 16] -Output [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Output [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] (73) Sort [codegen id : 9] -Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: [c_customer_sk#70 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#51] -Right keys [1]: [c_customer_sk#71] +Left keys [1]: [ss_customer_sk#50] +Right keys [1]: [c_customer_sk#70] Join condition: None (75) Project [codegen id : 10] -Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Input [16]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Output [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Input [16]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] (76) Exchange -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: hashpartitioning(ss_ticket_number#53, ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: hashpartitioning(ss_ticket_number#52, ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#74] (77) Sort [codegen id : 11] -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: [ss_ticket_number#53 ASC NULLS FIRST, ss_item_sk#50 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST], false, 0 (78) ReusedExchange [Reuses operator id: 26] -Output [2]: [sr_item_sk#76, sr_ticket_number#77] +Output [2]: [sr_item_sk#75, sr_ticket_number#76] (79) Sort [codegen id : 13] -Input [2]: [sr_item_sk#76, sr_ticket_number#77] -Arguments: [sr_ticket_number#77 ASC NULLS FIRST, sr_item_sk#76 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#75, sr_ticket_number#76] +Arguments: [sr_ticket_number#76 ASC NULLS FIRST, sr_item_sk#75 ASC NULLS FIRST], false, 0 (80) SortMergeJoin [codegen id : 14] -Left keys [2]: [ss_ticket_number#53, ss_item_sk#50] -Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] +Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] +Right keys [2]: [sr_ticket_number#76, sr_item_sk#75] Join condition: None (81) Project [codegen id : 14] -Output [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Input [16]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, sr_item_sk#76, sr_ticket_number#77] +Output [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Input [16]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, sr_item_sk#75, sr_ticket_number#76] (82) Exchange -Input [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: hashpartitioning(c_birth_country#74, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: hashpartitioning(c_birth_country#73, s_zip#59, 5), ENSURE_REQUIREMENTS, [id=#77] (83) Sort [codegen id : 15] -Input [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: [c_birth_country#74 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 +Input [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: [c_birth_country#73 ASC NULLS FIRST, s_zip#59 ASC NULLS FIRST], false, 0 (84) Scan parquet default.customer_address -Output [3]: [ca_state#79, ca_zip#80, ca_country#81] +Output [3]: [ca_state#78, ca_zip#79, ca_country#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (85) ColumnarToRow [codegen id : 16] -Input [3]: [ca_state#79, ca_zip#80, ca_country#81] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] (86) Filter [codegen id : 16] -Input [3]: [ca_state#79, ca_zip#80, ca_country#81] -Condition : ((isnotnull(ca_country#81) AND isnotnull(ca_zip#80)) AND isnotnull(upper(ca_country#81))) +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Condition : ((isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) AND isnotnull(upper(ca_country#80))) (87) Project [codegen id : 16] -Output [3]: [ca_state#79, ca_zip#80, upper(ca_country#81) AS upper(spark_catalog.default.customer_address.ca_country)#82] -Input [3]: [ca_state#79, ca_zip#80, ca_country#81] +Output [3]: [ca_state#78, ca_zip#79, upper(ca_country#80) AS upper(spark_catalog.default.customer_address.ca_country)#81] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] (88) Exchange -Input [3]: [ca_state#79, ca_zip#80, upper(spark_catalog.default.customer_address.ca_country)#82] -Arguments: hashpartitioning(upper(spark_catalog.default.customer_address.ca_country)#82, ca_zip#80, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [3]: [ca_state#78, ca_zip#79, upper(spark_catalog.default.customer_address.ca_country)#81] +Arguments: hashpartitioning(upper(spark_catalog.default.customer_address.ca_country)#81, ca_zip#79, 5), ENSURE_REQUIREMENTS, [id=#82] (89) Sort [codegen id : 17] -Input [3]: [ca_state#79, ca_zip#80, upper(spark_catalog.default.customer_address.ca_country)#82] -Arguments: [upper(spark_catalog.default.customer_address.ca_country)#82 ASC NULLS FIRST, ca_zip#80 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#78, ca_zip#79, upper(spark_catalog.default.customer_address.ca_country)#81] +Arguments: [upper(spark_catalog.default.customer_address.ca_country)#81 ASC NULLS FIRST, ca_zip#79 ASC NULLS FIRST], false, 0 (90) SortMergeJoin [codegen id : 18] -Left keys [2]: [c_birth_country#74, s_zip#60] -Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#82, ca_zip#80] +Left keys [2]: [c_birth_country#73, s_zip#59] +Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#81, ca_zip#79] Join condition: None (91) Project [codegen id : 18] -Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#79] -Input [15]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, ca_state#79, ca_zip#80, upper(spark_catalog.default.customer_address.ca_country)#82] +Output [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#78] +Input [15]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, ca_state#78, ca_zip#79, upper(spark_catalog.default.customer_address.ca_country)#81] (92) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#79] -Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum#84] -Results [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#85] +Input [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#78] +Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum#83] +Results [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#84] (93) Exchange -Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#85] -Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#84] +Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#85] (94) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#85] -Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#79, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] -Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#87] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#87,17,2) AS netpaid#40] +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#84] +Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] +Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#86] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#86,17,2) AS netpaid#39] (95) HashAggregate [codegen id : 19] -Input [1]: [netpaid#40] +Input [1]: [netpaid#39] Keys: [] -Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#88, count#89] -Results [2]: [sum#90, count#91] +Functions [1]: [partial_avg(netpaid#39)] +Aggregate Attributes [2]: [sum#87, count#88] +Results [2]: [sum#89, count#90] (96) Exchange -Input [2]: [sum#90, count#91] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#92] +Input [2]: [sum#89, count#90] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] (97) HashAggregate [codegen id : 20] -Input [2]: [sum#90, count#91] +Input [2]: [sum#89, count#90] Keys: [] -Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#93] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#93)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#94] +Functions [1]: [avg(netpaid#39)] +Aggregate Attributes [1]: [avg(netpaid#39)#92] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#92)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#93] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt index 5944b47e7a15d..bb65ace10c8e9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt @@ -101,7 +101,7 @@ WholeStageCodegen (14) WholeStageCodegen (12) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [ss_store_sk,c_birth_country,s_store_sk,upper(spark_catalog.default.customer_address.ca_country)] + BroadcastHashJoin [ss_store_sk,c_birth_country,s_store_sk,ca_country] Project [ss_store_sk,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt index c0e873fe39fc2..443ca4d35b780 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) +* Filter (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) :- * Project (33) : +- * BroadcastHashJoin Inner BuildRight (32) : :- * Project (27) @@ -41,11 +41,10 @@ : +- * Filter (30) : +- * ColumnarToRow (29) : +- Scan parquet default.customer (28) - +- BroadcastExchange (38) - +- * Project (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.customer_address (34) + +- BroadcastExchange (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.customer_address (34) (1) Scan parquet default.store_sales @@ -208,238 +207,211 @@ Input [3]: [ca_state#30, ca_zip#31, ca_country#32] Input [3]: [ca_state#30, ca_zip#31, ca_country#32] Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) -(37) Project [codegen id : 8] -Output [3]: [ca_state#30, ca_zip#31, upper(ca_country#32) AS upper(spark_catalog.default.customer_address.ca_country)#33] +(37) BroadcastExchange Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#33] -(38) BroadcastExchange -Input [3]: [ca_state#30, ca_zip#31, upper(spark_catalog.default.customer_address.ca_country)#33] -Arguments: HashedRelationBroadcastMode(List(input[2, string, true], input[1, string, true]),false), [id=#34] - -(39) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 9] Left keys [2]: [c_birth_country#28, s_zip#16] -Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#33, ca_zip#31] +Right keys [2]: [upper(ca_country#32), ca_zip#31] Join condition: None -(40) Project [codegen id : 9] +(39) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, ca_state#30] -Input [15]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, c_birth_country#28, ca_state#30, ca_zip#31, upper(spark_catalog.default.customer_address.ca_country)#33] +Input [15]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, c_birth_country#28, ca_state#30, ca_zip#31, ca_country#32] -(41) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, ca_state#30] Keys [10]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#35] -Results [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#36] +Aggregate Attributes [1]: [sum#34] +Results [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#35] -(42) Exchange -Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#36] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#37] +(41) Exchange +Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#35] +Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#36] -(43) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#36] +(42) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#35] Keys [10]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] -Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#37] +Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#37,17,2) AS netpaid#38] -(44) HashAggregate [codegen id : 10] -Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, netpaid#39] +(43) HashAggregate [codegen id : 10] +Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, netpaid#38] Keys [3]: [c_last_name#27, c_first_name#26, s_store_name#13] -Functions [1]: [partial_sum(netpaid#39)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#42, isEmpty#43] +Functions [1]: [partial_sum(netpaid#38)] +Aggregate Attributes [2]: [sum#39, isEmpty#40] +Results [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42] -(45) Exchange -Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#42, isEmpty#43] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#44] +(44) Exchange +Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42] +Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#43] -(46) HashAggregate [codegen id : 11] -Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#42, isEmpty#43] +(45) HashAggregate [codegen id : 11] +Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42] Keys [3]: [c_last_name#27, c_first_name#26, s_store_name#13] -Functions [1]: [sum(netpaid#39)] -Aggregate Attributes [1]: [sum(netpaid#39)#45] -Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, sum(netpaid#39)#45 AS paid#46] +Functions [1]: [sum(netpaid#38)] +Aggregate Attributes [1]: [sum(netpaid#38)#44] +Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, sum(netpaid#38)#44 AS paid#45] -(47) Filter [codegen id : 11] -Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) +(46) Filter [codegen id : 11] +Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#45] +Condition : (isnotnull(paid#45) AND (cast(paid#45 as decimal(33,8)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (78) -+- Exchange (77) - +- * HashAggregate (76) - +- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (56) - : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : :- * Project (53) - : : : : +- * SortMergeJoin Inner (52) - : : : : :- * Sort (49) - : : : : : +- ReusedExchange (48) - : : : : +- * Sort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- BroadcastExchange (60) - : : +- * Filter (59) - : : +- * ColumnarToRow (58) - : : +- Scan parquet default.item (57) - : +- ReusedExchange (63) - +- BroadcastExchange (70) - +- * Project (69) - +- * Filter (68) - +- * ColumnarToRow (67) - +- Scan parquet default.customer_address (66) - - -(48) ReusedExchange [Reuses operator id: 5] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) Sort [codegen id : 2] -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST], false, 0 - -(50) ReusedExchange [Reuses operator id: 11] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) Sort [codegen id : 4] -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST], false, 0 - -(52) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] -Right keys [2]: [sr_ticket_number#55, sr_item_sk#54] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#46, [id=#47] +* HashAggregate (73) ++- Exchange (72) + +- * HashAggregate (71) + +- * HashAggregate (70) + +- Exchange (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Project (52) + : : : : +- * SortMergeJoin Inner (51) + : : : : :- * Sort (48) + : : : : : +- ReusedExchange (47) + : : : : +- * Sort (50) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (59) + : : +- * Filter (58) + : : +- * ColumnarToRow (57) + : : +- Scan parquet default.item (56) + : +- ReusedExchange (62) + +- ReusedExchange (65) + + +(47) ReusedExchange [Reuses operator id: 5] +Output [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] + +(48) Sort [codegen id : 2] +Input [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] +Arguments: [ss_ticket_number#51 ASC NULLS FIRST, ss_item_sk#48 ASC NULLS FIRST], false, 0 + +(49) ReusedExchange [Reuses operator id: 11] +Output [2]: [sr_item_sk#53, sr_ticket_number#54] + +(50) Sort [codegen id : 4] +Input [2]: [sr_item_sk#53, sr_ticket_number#54] +Arguments: [sr_ticket_number#54 ASC NULLS FIRST, sr_item_sk#53 ASC NULLS FIRST], false, 0 + +(51) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#51, ss_item_sk#48] +Right keys [2]: [sr_ticket_number#54, sr_item_sk#53] Join condition: None -(53) Project [codegen id : 9] -Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] +(52) Project [codegen id : 9] +Output [4]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52] +Input [7]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, sr_item_sk#53, sr_ticket_number#54] -(54) ReusedExchange [Reuses operator id: 19] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] +(53) ReusedExchange [Reuses operator id: 19] +Output [4]: [s_store_sk#55, s_store_name#56, s_state#57, s_zip#58] -(55) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#56] +(54) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#50] +Right keys [1]: [s_store_sk#55] Join condition: None -(56) Project [codegen id : 9] -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] +(55) Project [codegen id : 9] +Output [6]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58] +Input [8]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52, s_store_sk#55, s_store_name#56, s_state#57, s_zip#58] -(57) Scan parquet default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] +(56) Scan parquet default.item +Output [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] +(57) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -(59) Filter [codegen id : 6] -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) +(58) Filter [codegen id : 6] +Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +Condition : isnotnull(i_item_sk#59) -(60) BroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] +(59) BroadcastExchange +Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] -(61) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#49] -Right keys [1]: [i_item_sk#60] +(60) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#48] +Right keys [1]: [i_item_sk#59] Join condition: None -(62) Project [codegen id : 9] -Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] +(61) Project [codegen id : 9] +Output [10]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +Input [12]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -(63) ReusedExchange [Reuses operator id: 31] -Output [4]: [c_customer_sk#67, c_first_name#68, c_last_name#69, c_birth_country#70] +(62) ReusedExchange [Reuses operator id: 31] +Output [4]: [c_customer_sk#66, c_first_name#67, c_last_name#68, c_birth_country#69] -(64) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#50] -Right keys [1]: [c_customer_sk#67] +(63) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#49] +Right keys [1]: [c_customer_sk#66] Join condition: None -(65) Project [codegen id : 9] -Output [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, c_birth_country#70] -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_customer_sk#67, c_first_name#68, c_last_name#69, c_birth_country#70] - -(66) Scan parquet default.customer_address -Output [3]: [ca_state#71, ca_zip#72, ca_country#73] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] -ReadSchema: struct - -(67) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#71, ca_zip#72, ca_country#73] - -(68) Filter [codegen id : 8] -Input [3]: [ca_state#71, ca_zip#72, ca_country#73] -Condition : ((isnotnull(ca_country#73) AND isnotnull(ca_zip#72)) AND isnotnull(upper(ca_country#73))) - -(69) Project [codegen id : 8] -Output [3]: [ca_state#71, ca_zip#72, upper(ca_country#73) AS upper(spark_catalog.default.customer_address.ca_country)#74] -Input [3]: [ca_state#71, ca_zip#72, ca_country#73] +(64) Project [codegen id : 9] +Output [12]: [ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, c_birth_country#69] +Input [14]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_customer_sk#66, c_first_name#67, c_last_name#68, c_birth_country#69] -(70) BroadcastExchange -Input [3]: [ca_state#71, ca_zip#72, upper(spark_catalog.default.customer_address.ca_country)#74] -Arguments: HashedRelationBroadcastMode(List(input[2, string, true], input[1, string, true]),false), [id=#75] +(65) ReusedExchange [Reuses operator id: 37] +Output [3]: [ca_state#70, ca_zip#71, ca_country#72] -(71) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#70, s_zip#59] -Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#74, ca_zip#72] +(66) BroadcastHashJoin [codegen id : 9] +Left keys [2]: [c_birth_country#69, s_zip#58] +Right keys [2]: [upper(ca_country#72), ca_zip#71] Join condition: None -(72) Project [codegen id : 9] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, ca_state#71] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, c_birth_country#70, ca_state#71, ca_zip#72, upper(spark_catalog.default.customer_address.ca_country)#74] - -(73) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65, c_first_name#68, c_last_name#69, ca_state#71] -Keys [10]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#76] -Results [11]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, sum#77] - -(74) Exchange -Input [11]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, sum#77] -Arguments: hashpartitioning(c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, 5), ENSURE_REQUIREMENTS, [id=#78] - -(75) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62, sum#77] -Keys [10]: [c_last_name#69, c_first_name#68, s_store_name#57, ca_state#71, s_state#58, i_color#63, i_current_price#61, i_manager_id#65, i_units#64, i_size#62] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#79] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#79,17,2) AS netpaid#39] - -(76) HashAggregate [codegen id : 10] -Input [1]: [netpaid#39] +(67) Project [codegen id : 9] +Output [11]: [ss_net_paid#52, s_store_name#56, s_state#57, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, ca_state#70] +Input [15]: [ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, c_birth_country#69, ca_state#70, ca_zip#71, ca_country#72] + +(68) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#52, s_store_name#56, s_state#57, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, ca_state#70] +Keys [10]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [sum#73] +Results [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] + +(69) Exchange +Input [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] +Arguments: hashpartitioning(c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, 5), ENSURE_REQUIREMENTS, [id=#75] + +(70) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] +Keys [10]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61] +Functions [1]: [sum(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#52))#76] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#52))#76,17,2) AS netpaid#38] + +(71) HashAggregate [codegen id : 10] +Input [1]: [netpaid#38] Keys: [] -Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#80, count#81] -Results [2]: [sum#82, count#83] +Functions [1]: [partial_avg(netpaid#38)] +Aggregate Attributes [2]: [sum#77, count#78] +Results [2]: [sum#79, count#80] -(77) Exchange -Input [2]: [sum#82, count#83] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#84] +(72) Exchange +Input [2]: [sum#79, count#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] -(78) HashAggregate [codegen id : 11] -Input [2]: [sum#82, count#83] +(73) HashAggregate [codegen id : 11] +Input [2]: [sum#79, count#80] Keys: [] -Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#85] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#85)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#86] +Functions [1]: [avg(netpaid#38)] +Aggregate Attributes [1]: [avg(netpaid#38)#82] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#82)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#83] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index f60055c66c5cb..2d9ef020540b8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -13,7 +13,7 @@ WholeStageCodegen (11) WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,upper(spark_catalog.default.customer_address.ca_country),ca_zip] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -44,13 +44,7 @@ WholeStageCodegen (11) InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (8) - Project [ca_state,ca_zip,ca_country] - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + ReusedExchange [ca_state,ca_zip,ca_country] #8 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name,s_store_name] #1 @@ -62,7 +56,7 @@ WholeStageCodegen (11) WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,upper(spark_catalog.default.customer_address.ca_country),ca_zip] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -118,8 +112,7 @@ WholeStageCodegen (11) InputAdapter BroadcastExchange #8 WholeStageCodegen (8) - Project [ca_state,ca_zip,ca_country] - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + Filter [ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt index 9a243e2105c34..7db465072e4c3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt @@ -199,112 +199,112 @@ Right keys [1]: [ctr_customer_sk#33] Join condition: None (32) Project [codegen id : 17] -Output [15]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_state#34, ctr_total_return#35, cast(ctr_total_return#35 as decimal(24,7)) AS CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#36] +Output [14]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_state#34, ctr_total_return#35] Input [16]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_customer_sk#33, ctr_state#34, ctr_total_return#35] (33) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39, wr_returned_date_sk#40] +Output [4]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, wr_returned_date_sk#39] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#40), dynamicpruningexpression(wr_returned_date_sk#40 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(wr_returned_date_sk#39), dynamicpruningexpression(wr_returned_date_sk#39 IN dynamicpruning#23)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 10] -Input [4]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39, wr_returned_date_sk#40] +Input [4]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, wr_returned_date_sk#39] (35) Filter [codegen id : 10] -Input [4]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39, wr_returned_date_sk#40] -Condition : isnotnull(wr_returning_addr_sk#38) +Input [4]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, wr_returned_date_sk#39] +Condition : isnotnull(wr_returning_addr_sk#37) (36) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#41] +Output [1]: [d_date_sk#40] (37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [wr_returned_date_sk#40] -Right keys [1]: [d_date_sk#41] +Left keys [1]: [wr_returned_date_sk#39] +Right keys [1]: [d_date_sk#40] Join condition: None (38) Project [codegen id : 10] -Output [3]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39] -Input [5]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39, wr_returned_date_sk#40, d_date_sk#41] +Output [3]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38] +Input [5]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, wr_returned_date_sk#39, d_date_sk#40] (39) Exchange -Input [3]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39] -Arguments: hashpartitioning(wr_returning_addr_sk#38, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38] +Arguments: hashpartitioning(wr_returning_addr_sk#37, 5), ENSURE_REQUIREMENTS, [id=#41] (40) Sort [codegen id : 11] -Input [3]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39] -Arguments: [wr_returning_addr_sk#38 ASC NULLS FIRST], false, 0 +Input [3]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38] +Arguments: [wr_returning_addr_sk#37 ASC NULLS FIRST], false, 0 (41) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#43, ca_state#44] +Output [2]: [ca_address_sk#42, ca_state#43] (42) Sort [codegen id : 13] -Input [2]: [ca_address_sk#43, ca_state#44] -Arguments: [ca_address_sk#43 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#42, ca_state#43] +Arguments: [ca_address_sk#42 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 14] -Left keys [1]: [wr_returning_addr_sk#38] -Right keys [1]: [ca_address_sk#43] +Left keys [1]: [wr_returning_addr_sk#37] +Right keys [1]: [ca_address_sk#42] Join condition: None (44) Project [codegen id : 14] -Output [3]: [wr_returning_customer_sk#37, wr_return_amt#39, ca_state#44] -Input [5]: [wr_returning_customer_sk#37, wr_returning_addr_sk#38, wr_return_amt#39, ca_address_sk#43, ca_state#44] +Output [3]: [wr_returning_customer_sk#36, wr_return_amt#38, ca_state#43] +Input [5]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, ca_address_sk#42, ca_state#43] (45) HashAggregate [codegen id : 14] -Input [3]: [wr_returning_customer_sk#37, wr_return_amt#39, ca_state#44] -Keys [2]: [wr_returning_customer_sk#37, ca_state#44] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#39))] -Aggregate Attributes [1]: [sum#45] -Results [3]: [wr_returning_customer_sk#37, ca_state#44, sum#46] +Input [3]: [wr_returning_customer_sk#36, wr_return_amt#38, ca_state#43] +Keys [2]: [wr_returning_customer_sk#36, ca_state#43] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#38))] +Aggregate Attributes [1]: [sum#44] +Results [3]: [wr_returning_customer_sk#36, ca_state#43, sum#45] (46) Exchange -Input [3]: [wr_returning_customer_sk#37, ca_state#44, sum#46] -Arguments: hashpartitioning(wr_returning_customer_sk#37, ca_state#44, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [3]: [wr_returning_customer_sk#36, ca_state#43, sum#45] +Arguments: hashpartitioning(wr_returning_customer_sk#36, ca_state#43, 5), ENSURE_REQUIREMENTS, [id=#46] (47) HashAggregate [codegen id : 15] -Input [3]: [wr_returning_customer_sk#37, ca_state#44, sum#46] -Keys [2]: [wr_returning_customer_sk#37, ca_state#44] -Functions [1]: [sum(UnscaledValue(wr_return_amt#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#39))#48] -Results [2]: [ca_state#44 AS ctr_state#34, MakeDecimal(sum(UnscaledValue(wr_return_amt#39))#48,17,2) AS ctr_total_return#35] +Input [3]: [wr_returning_customer_sk#36, ca_state#43, sum#45] +Keys [2]: [wr_returning_customer_sk#36, ca_state#43] +Functions [1]: [sum(UnscaledValue(wr_return_amt#38))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#38))#47] +Results [2]: [ca_state#43 AS ctr_state#34, MakeDecimal(sum(UnscaledValue(wr_return_amt#38))#47,17,2) AS ctr_total_return#35] (48) HashAggregate [codegen id : 15] Input [2]: [ctr_state#34, ctr_total_return#35] Keys [1]: [ctr_state#34] Functions [1]: [partial_avg(ctr_total_return#35)] -Aggregate Attributes [2]: [sum#49, count#50] -Results [3]: [ctr_state#34, sum#51, count#52] +Aggregate Attributes [2]: [sum#48, count#49] +Results [3]: [ctr_state#34, sum#50, count#51] (49) Exchange -Input [3]: [ctr_state#34, sum#51, count#52] -Arguments: hashpartitioning(ctr_state#34, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [3]: [ctr_state#34, sum#50, count#51] +Arguments: hashpartitioning(ctr_state#34, 5), ENSURE_REQUIREMENTS, [id=#52] (50) HashAggregate [codegen id : 16] -Input [3]: [ctr_state#34, sum#51, count#52] +Input [3]: [ctr_state#34, sum#50, count#51] Keys [1]: [ctr_state#34] Functions [1]: [avg(ctr_total_return#35)] -Aggregate Attributes [1]: [avg(ctr_total_return#35)#54] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#35)#54) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#55, ctr_state#34 AS ctr_state#34#56] +Aggregate Attributes [1]: [avg(ctr_total_return#35)#53] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#35)#53) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#54, ctr_state#34 AS ctr_state#34#55] (51) Filter [codegen id : 16] -Input [2]: [(avg(ctr_total_return) * 1.2)#55, ctr_state#34#56] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#55) +Input [2]: [(avg(ctr_total_return) * 1.2)#54, ctr_state#34#55] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#54) (52) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#55, ctr_state#34#56] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#57] +Input [2]: [(avg(ctr_total_return) * 1.2)#54, ctr_state#34#55] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#56] (53) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ctr_state#34] -Right keys [1]: [ctr_state#34#56] -Join condition: (CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#36 > (avg(ctr_total_return) * 1.2)#55) +Right keys [1]: [ctr_state#34#55] +Join condition: (cast(ctr_total_return#35 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#54) (54) Project [codegen id : 17] Output [13]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_total_return#35] -Input [17]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_state#34, ctr_total_return#35, CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#36, (avg(ctr_total_return) * 1.2)#55, ctr_state#34#56] +Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_state#34, ctr_total_return#35, (avg(ctr_total_return) * 1.2)#54, ctr_state#34#55] (55) TakeOrderedAndProject Input [13]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_total_return#35] @@ -321,27 +321,27 @@ BroadcastExchange (60) (56) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_year#58] +Output [2]: [d_date_sk#24, d_year#57] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#58] +Input [2]: [d_date_sk#24, d_year#57] (58) Filter [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#58] -Condition : ((isnotnull(d_year#58) AND (d_year#58 = 2002)) AND isnotnull(d_date_sk#24)) +Input [2]: [d_date_sk#24, d_year#57] +Condition : ((isnotnull(d_year#57) AND (d_year#57 = 2002)) AND isnotnull(d_date_sk#24)) (59) Project [codegen id : 1] Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#58] +Input [2]: [d_date_sk#24, d_year#57] (60) BroadcastExchange Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] -Subquery:2 Hosting operator id = 33 Hosting Expression = wr_returned_date_sk#40 IN dynamicpruning#23 +Subquery:2 Hosting operator id = 33 Hosting Expression = wr_returned_date_sk#39 IN dynamicpruning#23 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/simplified.txt index 2c51d1c8b349e..6983e407ba891 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] WholeStageCodegen (17) Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,CAST(ctr1.ctr_total_return AS DECIMAL(24,7)),(avg(ctr_total_return) * 1.2)] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_state,ctr_total_return] BroadcastHashJoin [c_customer_sk,ctr_customer_sk] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt index 163edf751ab1b..066b0fa66bf33 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt @@ -1,54 +1,53 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Project (17) - : : : +- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.web_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.customer_address (7) - : : +- BroadcastExchange (34) - : : +- * Filter (33) - : : +- * HashAggregate (32) - : : +- Exchange (31) - : : +- * HashAggregate (30) - : : +- * HashAggregate (29) - : : +- Exchange (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * Filter (20) - : : : : +- * ColumnarToRow (19) - : : : : +- Scan parquet default.web_returns (18) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- BroadcastExchange (40) - : +- * Filter (39) - : +- * ColumnarToRow (38) - : +- Scan parquet default.customer (37) - +- BroadcastExchange (47) - +- * Project (46) - +- * Filter (45) - +- * ColumnarToRow (44) - +- Scan parquet default.customer_address (43) +TakeOrderedAndProject (49) ++- * Project (48) + +- * BroadcastHashJoin Inner BuildRight (47) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.web_returns (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.customer_address (7) + : : +- BroadcastExchange (33) + : : +- * Filter (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * HashAggregate (28) + : : +- Exchange (27) + : : +- * HashAggregate (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * Filter (19) + : : : : +- * ColumnarToRow (18) + : : : : +- Scan parquet default.web_returns (17) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : +- BroadcastExchange (39) + : +- * Filter (38) + : +- * ColumnarToRow (37) + : +- Scan parquet default.customer (36) + +- BroadcastExchange (46) + +- * Project (45) + +- * Filter (44) + +- * ColumnarToRow (43) + +- Scan parquet default.customer_address (42) (1) Scan parquet default.web_returns @@ -66,7 +65,7 @@ Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) -(4) ReusedExchange [Reuses operator id: 55] +(4) ReusedExchange [Reuses operator id: 54] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -127,196 +126,192 @@ Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#14, ca_state#8 AS ct Input [3]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16] Condition : isnotnull(ctr_total_return#16) -(17) Project [codegen id : 11] -Output [4]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, cast(ctr_total_return#16 as decimal(24,7)) AS CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#17] -Input [3]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16] - -(18) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#18, wr_returning_addr_sk#19, wr_return_amt#20, wr_returned_date_sk#21] +(17) Scan parquet default.web_returns +Output [4]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, wr_returned_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#21), dynamicpruningexpression(wr_returned_date_sk#21 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#20), dynamicpruningexpression(wr_returned_date_sk#20 IN dynamicpruning#5)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returning_customer_sk#18, wr_returning_addr_sk#19, wr_return_amt#20, wr_returned_date_sk#21] +(18) ColumnarToRow [codegen id : 6] +Input [4]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, wr_returned_date_sk#20] -(20) Filter [codegen id : 6] -Input [4]: [wr_returning_customer_sk#18, wr_returning_addr_sk#19, wr_return_amt#20, wr_returned_date_sk#21] -Condition : isnotnull(wr_returning_addr_sk#19) +(19) Filter [codegen id : 6] +Input [4]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, wr_returned_date_sk#20] +Condition : isnotnull(wr_returning_addr_sk#18) -(21) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#22] +(20) ReusedExchange [Reuses operator id: 54] +Output [1]: [d_date_sk#21] -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#21] -Right keys [1]: [d_date_sk#22] +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [wr_returned_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None -(23) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#18, wr_returning_addr_sk#19, wr_return_amt#20] -Input [5]: [wr_returning_customer_sk#18, wr_returning_addr_sk#19, wr_return_amt#20, wr_returned_date_sk#21, d_date_sk#22] +(22) Project [codegen id : 6] +Output [3]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19] +Input [5]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, wr_returned_date_sk#20, d_date_sk#21] -(24) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#23, ca_state#24] +(23) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#22, ca_state#23] -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#19] -Right keys [1]: [ca_address_sk#23] +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [wr_returning_addr_sk#18] +Right keys [1]: [ca_address_sk#22] Join condition: None -(26) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#18, wr_return_amt#20, ca_state#24] -Input [5]: [wr_returning_customer_sk#18, wr_returning_addr_sk#19, wr_return_amt#20, ca_address_sk#23, ca_state#24] +(25) Project [codegen id : 6] +Output [3]: [wr_returning_customer_sk#17, wr_return_amt#19, ca_state#23] +Input [5]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, ca_address_sk#22, ca_state#23] -(27) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#18, wr_return_amt#20, ca_state#24] -Keys [2]: [wr_returning_customer_sk#18, ca_state#24] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#20))] -Aggregate Attributes [1]: [sum#25] -Results [3]: [wr_returning_customer_sk#18, ca_state#24, sum#26] +(26) HashAggregate [codegen id : 6] +Input [3]: [wr_returning_customer_sk#17, wr_return_amt#19, ca_state#23] +Keys [2]: [wr_returning_customer_sk#17, ca_state#23] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#19))] +Aggregate Attributes [1]: [sum#24] +Results [3]: [wr_returning_customer_sk#17, ca_state#23, sum#25] -(28) Exchange -Input [3]: [wr_returning_customer_sk#18, ca_state#24, sum#26] -Arguments: hashpartitioning(wr_returning_customer_sk#18, ca_state#24, 5), ENSURE_REQUIREMENTS, [id=#27] +(27) Exchange +Input [3]: [wr_returning_customer_sk#17, ca_state#23, sum#25] +Arguments: hashpartitioning(wr_returning_customer_sk#17, ca_state#23, 5), ENSURE_REQUIREMENTS, [id=#26] -(29) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#18, ca_state#24, sum#26] -Keys [2]: [wr_returning_customer_sk#18, ca_state#24] -Functions [1]: [sum(UnscaledValue(wr_return_amt#20))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#20))#28] -Results [2]: [ca_state#24 AS ctr_state#15, MakeDecimal(sum(UnscaledValue(wr_return_amt#20))#28,17,2) AS ctr_total_return#16] +(28) HashAggregate [codegen id : 7] +Input [3]: [wr_returning_customer_sk#17, ca_state#23, sum#25] +Keys [2]: [wr_returning_customer_sk#17, ca_state#23] +Functions [1]: [sum(UnscaledValue(wr_return_amt#19))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#19))#27] +Results [2]: [ca_state#23 AS ctr_state#15, MakeDecimal(sum(UnscaledValue(wr_return_amt#19))#27,17,2) AS ctr_total_return#16] -(30) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 7] Input [2]: [ctr_state#15, ctr_total_return#16] Keys [1]: [ctr_state#15] Functions [1]: [partial_avg(ctr_total_return#16)] -Aggregate Attributes [2]: [sum#29, count#30] -Results [3]: [ctr_state#15, sum#31, count#32] +Aggregate Attributes [2]: [sum#28, count#29] +Results [3]: [ctr_state#15, sum#30, count#31] -(31) Exchange -Input [3]: [ctr_state#15, sum#31, count#32] -Arguments: hashpartitioning(ctr_state#15, 5), ENSURE_REQUIREMENTS, [id=#33] +(30) Exchange +Input [3]: [ctr_state#15, sum#30, count#31] +Arguments: hashpartitioning(ctr_state#15, 5), ENSURE_REQUIREMENTS, [id=#32] -(32) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#15, sum#31, count#32] +(31) HashAggregate [codegen id : 8] +Input [3]: [ctr_state#15, sum#30, count#31] Keys [1]: [ctr_state#15] Functions [1]: [avg(ctr_total_return#16)] -Aggregate Attributes [1]: [avg(ctr_total_return#16)#34] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#16)#34) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#35, ctr_state#15 AS ctr_state#15#36] +Aggregate Attributes [1]: [avg(ctr_total_return#16)#33] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#16)#33) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#34, ctr_state#15 AS ctr_state#15#35] -(33) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#35, ctr_state#15#36] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#35) +(32) Filter [codegen id : 8] +Input [2]: [(avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#34) -(34) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#35, ctr_state#15#36] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#37] +(33) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#36] -(35) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#15] -Right keys [1]: [ctr_state#15#36] -Join condition: (CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#17 > (avg(ctr_total_return) * 1.2)#35) +Right keys [1]: [ctr_state#15#35] +Join condition: (cast(ctr_total_return#16 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#34) -(36) Project [codegen id : 11] +(35) Project [codegen id : 11] Output [2]: [ctr_customer_sk#14, ctr_total_return#16] -Input [6]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#17, (avg(ctr_total_return) * 1.2)#35, ctr_state#15#36] +Input [5]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, (avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] -(37) Scan parquet default.customer -Output [14]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51] +(36) Scan parquet default.customer +Output [14]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51] +(37) ColumnarToRow [codegen id : 9] +Input [14]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] -(39) Filter [codegen id : 9] -Input [14]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51] -Condition : (isnotnull(c_customer_sk#38) AND isnotnull(c_current_addr_sk#40)) +(38) Filter [codegen id : 9] +Input [14]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] +Condition : (isnotnull(c_customer_sk#37) AND isnotnull(c_current_addr_sk#39)) -(40) BroadcastExchange -Input [14]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +(39) BroadcastExchange +Input [14]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] -(41) BroadcastHashJoin [codegen id : 11] +(40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#14] -Right keys [1]: [c_customer_sk#38] +Right keys [1]: [c_customer_sk#37] Join condition: None -(42) Project [codegen id : 11] -Output [14]: [ctr_total_return#16, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51] -Input [16]: [ctr_customer_sk#14, ctr_total_return#16, c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51] +(41) Project [codegen id : 11] +Output [14]: [ctr_total_return#16, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] +Input [16]: [ctr_customer_sk#14, ctr_total_return#16, c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] -(43) Scan parquet default.customer_address -Output [2]: [ca_address_sk#53, ca_state#54] +(42) Scan parquet default.customer_address +Output [2]: [ca_address_sk#52, ca_state#53] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 10] -Input [2]: [ca_address_sk#53, ca_state#54] +(43) ColumnarToRow [codegen id : 10] +Input [2]: [ca_address_sk#52, ca_state#53] -(45) Filter [codegen id : 10] -Input [2]: [ca_address_sk#53, ca_state#54] -Condition : ((isnotnull(ca_state#54) AND (ca_state#54 = GA)) AND isnotnull(ca_address_sk#53)) +(44) Filter [codegen id : 10] +Input [2]: [ca_address_sk#52, ca_state#53] +Condition : ((isnotnull(ca_state#53) AND (ca_state#53 = GA)) AND isnotnull(ca_address_sk#52)) -(46) Project [codegen id : 10] -Output [1]: [ca_address_sk#53] -Input [2]: [ca_address_sk#53, ca_state#54] +(45) Project [codegen id : 10] +Output [1]: [ca_address_sk#52] +Input [2]: [ca_address_sk#52, ca_state#53] -(47) BroadcastExchange -Input [1]: [ca_address_sk#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#55] +(46) BroadcastExchange +Input [1]: [ca_address_sk#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] -(48) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#53] +(47) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#39] +Right keys [1]: [ca_address_sk#52] Join condition: None -(49) Project [codegen id : 11] -Output [13]: [c_customer_id#39, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51, ctr_total_return#16] -Input [15]: [ctr_total_return#16, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51, ca_address_sk#53] +(48) Project [codegen id : 11] +Output [13]: [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50, ctr_total_return#16] +Input [15]: [ctr_total_return#16, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50, ca_address_sk#52] -(50) TakeOrderedAndProject -Input [13]: [c_customer_id#39, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51, ctr_total_return#16] -Arguments: 100, [c_customer_id#39 ASC NULLS FIRST, c_salutation#41 ASC NULLS FIRST, c_first_name#42 ASC NULLS FIRST, c_last_name#43 ASC NULLS FIRST, c_preferred_cust_flag#44 ASC NULLS FIRST, c_birth_day#45 ASC NULLS FIRST, c_birth_month#46 ASC NULLS FIRST, c_birth_year#47 ASC NULLS FIRST, c_birth_country#48 ASC NULLS FIRST, c_login#49 ASC NULLS FIRST, c_email_address#50 ASC NULLS FIRST, c_last_review_date#51 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#39, c_salutation#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_day#45, c_birth_month#46, c_birth_year#47, c_birth_country#48, c_login#49, c_email_address#50, c_last_review_date#51, ctr_total_return#16] +(49) TakeOrderedAndProject +Input [13]: [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50, ctr_total_return#16] +Arguments: 100, [c_customer_id#38 ASC NULLS FIRST, c_salutation#40 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, c_last_name#42 ASC NULLS FIRST, c_preferred_cust_flag#43 ASC NULLS FIRST, c_birth_day#44 ASC NULLS FIRST, c_birth_month#45 ASC NULLS FIRST, c_birth_year#46 ASC NULLS FIRST, c_birth_country#47 ASC NULLS FIRST, c_login#48 ASC NULLS FIRST, c_email_address#49 ASC NULLS FIRST, c_last_review_date#50 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50, ctr_total_return#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * Project (54) - +- * Filter (53) - +- * ColumnarToRow (52) - +- Scan parquet default.date_dim (51) +BroadcastExchange (54) ++- * Project (53) + +- * Filter (52) + +- * ColumnarToRow (51) + +- Scan parquet default.date_dim (50) -(51) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#56] +(50) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_year#55] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#56] +(51) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#55] -(53) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#56] -Condition : ((isnotnull(d_year#56) AND (d_year#56 = 2002)) AND isnotnull(d_date_sk#6)) +(52) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#55] +Condition : ((isnotnull(d_year#55) AND (d_year#55 = 2002)) AND isnotnull(d_date_sk#6)) -(54) Project [codegen id : 1] +(53) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#56] +Input [2]: [d_date_sk#6, d_year#55] -(55) BroadcastExchange +(54) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] -Subquery:2 Hosting operator id = 18 Hosting Expression = wr_returned_date_sk#21 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = wr_returned_date_sk#20 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt index dc3b631040b2d..884dce2c6583a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -5,39 +5,38 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,CAST(ctr1.ctr_total_return AS DECIMAL(24,7)),(avg(ctr_total_return) * 1.2)] - Project [ctr_customer_sk,ctr_state,ctr_total_return] - Filter [ctr_total_return] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [wr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_returning_addr_sk,wr_returning_customer_sk] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] + Filter [ctr_total_return] + HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] + InputAdapter + Exchange [wr_returning_customer_sk,ca_state] #1 + WholeStageCodegen (3) + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + Project [wr_returning_customer_sk,wr_return_amt,ca_state] + BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + Filter [wr_returning_addr_sk,wr_returning_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Filter [ca_address_sk,ca_state] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [ca_address_sk,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt index cba3fdedc086c..a9569e31b039f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt @@ -356,261 +356,261 @@ Right keys [1]: [ca_county#42] Join condition: None (54) Project [codegen id : 42] -Output [7]: [store_sales#16, store_sales#31, ca_county#42, d_year#38, store_sales#47, CASE WHEN (store_sales#47 > 0.00) THEN CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#47)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#49, CASE WHEN (store_sales#16 > 0.00) THEN CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#50] +Output [5]: [store_sales#16, store_sales#31, ca_county#42, d_year#38, store_sales#47] Input [6]: [ca_county#10, store_sales#16, store_sales#31, ca_county#42, d_year#38, store_sales#47] (55) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] +Output [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] +Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (57) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_bill_addr_sk#51) +Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_bill_addr_sk#49) (58) ReusedExchange [Reuses operator id: 122] -Output [3]: [d_date_sk#54, d_year#55, d_qoy#56] +Output [3]: [d_date_sk#52, d_year#53, d_qoy#54] (59) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#53] -Right keys [1]: [d_date_sk#54] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#52] Join condition: None (60) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#55, d_qoy#56] -Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53, d_date_sk#54, d_year#55, d_qoy#56] +Output [4]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54] +Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51, d_date_sk#52, d_year#53, d_qoy#54] (61) Exchange -Input [4]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#55, d_qoy#56] -Arguments: hashpartitioning(ws_bill_addr_sk#51, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [4]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54] +Arguments: hashpartitioning(ws_bill_addr_sk#49, 5), ENSURE_REQUIREMENTS, [id=#55] (62) Sort [codegen id : 23] -Input [4]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#55, d_qoy#56] -Arguments: [ws_bill_addr_sk#51 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54] +Arguments: [ws_bill_addr_sk#49 ASC NULLS FIRST], false, 0 (63) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#58, ca_county#59] +Output [2]: [ca_address_sk#56, ca_county#57] (64) Sort [codegen id : 25] -Input [2]: [ca_address_sk#58, ca_county#59] -Arguments: [ca_address_sk#58 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#56, ca_county#57] +Arguments: [ca_address_sk#56 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] -Left keys [1]: [ws_bill_addr_sk#51] -Right keys [1]: [ca_address_sk#58] +Left keys [1]: [ws_bill_addr_sk#49] +Right keys [1]: [ca_address_sk#56] Join condition: None (66) Project [codegen id : 26] -Output [4]: [ws_ext_sales_price#52, d_year#55, d_qoy#56, ca_county#59] -Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#55, d_qoy#56, ca_address_sk#58, ca_county#59] +Output [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#57] +Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_address_sk#56, ca_county#57] (67) HashAggregate [codegen id : 26] -Input [4]: [ws_ext_sales_price#52, d_year#55, d_qoy#56, ca_county#59] -Keys [3]: [ca_county#59, d_qoy#56, d_year#55] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#52))] -Aggregate Attributes [1]: [sum#60] -Results [4]: [ca_county#59, d_qoy#56, d_year#55, sum#61] +Input [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#57] +Keys [3]: [ca_county#57, d_qoy#54, d_year#53] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#50))] +Aggregate Attributes [1]: [sum#58] +Results [4]: [ca_county#57, d_qoy#54, d_year#53, sum#59] (68) Exchange -Input [4]: [ca_county#59, d_qoy#56, d_year#55, sum#61] -Arguments: hashpartitioning(ca_county#59, d_qoy#56, d_year#55, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [ca_county#57, d_qoy#54, d_year#53, sum#59] +Arguments: hashpartitioning(ca_county#57, d_qoy#54, d_year#53, 5), ENSURE_REQUIREMENTS, [id=#60] (69) HashAggregate [codegen id : 41] -Input [4]: [ca_county#59, d_qoy#56, d_year#55, sum#61] -Keys [3]: [ca_county#59, d_qoy#56, d_year#55] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#52))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#52))#63] -Results [2]: [ca_county#59, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#52))#63,17,2) AS web_sales#64] +Input [4]: [ca_county#57, d_qoy#54, d_year#53, sum#59] +Keys [3]: [ca_county#57, d_qoy#54, d_year#53] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#50))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#50))#61] +Results [2]: [ca_county#57, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#50))#61,17,2) AS web_sales#62] (70) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, ws_sold_date_sk#67] +Output [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#67), dynamicpruningexpression(ws_sold_date_sk#67 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (71) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, ws_sold_date_sk#67] +Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] (72) Filter [codegen id : 28] -Input [3]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, ws_sold_date_sk#67] -Condition : isnotnull(ws_bill_addr_sk#65) +Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Condition : isnotnull(ws_bill_addr_sk#63) (73) ReusedExchange [Reuses operator id: 114] -Output [3]: [d_date_sk#68, d_year#69, d_qoy#70] +Output [3]: [d_date_sk#66, d_year#67, d_qoy#68] (74) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#67] -Right keys [1]: [d_date_sk#68] +Left keys [1]: [ws_sold_date_sk#65] +Right keys [1]: [d_date_sk#66] Join condition: None (75) Project [codegen id : 28] -Output [4]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, d_year#69, d_qoy#70] -Input [6]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, ws_sold_date_sk#67, d_date_sk#68, d_year#69, d_qoy#70] +Output [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68] +Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65, d_date_sk#66, d_year#67, d_qoy#68] (76) Exchange -Input [4]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, d_year#69, d_qoy#70] -Arguments: hashpartitioning(ws_bill_addr_sk#65, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68] +Arguments: hashpartitioning(ws_bill_addr_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] (77) Sort [codegen id : 29] -Input [4]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, d_year#69, d_qoy#70] -Arguments: [ws_bill_addr_sk#65 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68] +Arguments: [ws_bill_addr_sk#63 ASC NULLS FIRST], false, 0 (78) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#72, ca_county#73] +Output [2]: [ca_address_sk#70, ca_county#71] (79) Sort [codegen id : 31] -Input [2]: [ca_address_sk#72, ca_county#73] -Arguments: [ca_address_sk#72 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#70, ca_county#71] +Arguments: [ca_address_sk#70 ASC NULLS FIRST], false, 0 (80) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_addr_sk#65] -Right keys [1]: [ca_address_sk#72] +Left keys [1]: [ws_bill_addr_sk#63] +Right keys [1]: [ca_address_sk#70] Join condition: None (81) Project [codegen id : 32] -Output [4]: [ws_ext_sales_price#66, d_year#69, d_qoy#70, ca_county#73] -Input [6]: [ws_bill_addr_sk#65, ws_ext_sales_price#66, d_year#69, d_qoy#70, ca_address_sk#72, ca_county#73] +Output [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#71] +Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_address_sk#70, ca_county#71] (82) HashAggregate [codegen id : 32] -Input [4]: [ws_ext_sales_price#66, d_year#69, d_qoy#70, ca_county#73] -Keys [3]: [ca_county#73, d_qoy#70, d_year#69] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#66))] -Aggregate Attributes [1]: [sum#74] -Results [4]: [ca_county#73, d_qoy#70, d_year#69, sum#75] +Input [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#71] +Keys [3]: [ca_county#71, d_qoy#68, d_year#67] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] +Aggregate Attributes [1]: [sum#72] +Results [4]: [ca_county#71, d_qoy#68, d_year#67, sum#73] (83) Exchange -Input [4]: [ca_county#73, d_qoy#70, d_year#69, sum#75] -Arguments: hashpartitioning(ca_county#73, d_qoy#70, d_year#69, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [4]: [ca_county#71, d_qoy#68, d_year#67, sum#73] +Arguments: hashpartitioning(ca_county#71, d_qoy#68, d_year#67, 5), ENSURE_REQUIREMENTS, [id=#74] (84) HashAggregate [codegen id : 33] -Input [4]: [ca_county#73, d_qoy#70, d_year#69, sum#75] -Keys [3]: [ca_county#73, d_qoy#70, d_year#69] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#66))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#66))#77] -Results [2]: [ca_county#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#77,17,2) AS web_sales#78] +Input [4]: [ca_county#71, d_qoy#68, d_year#67, sum#73] +Keys [3]: [ca_county#71, d_qoy#68, d_year#67] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#64))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#64))#75] +Results [2]: [ca_county#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#75,17,2) AS web_sales#76] (85) BroadcastExchange -Input [2]: [ca_county#73, web_sales#78] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#79] +Input [2]: [ca_county#71, web_sales#76] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#77] (86) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#59] -Right keys [1]: [ca_county#73] +Left keys [1]: [ca_county#57] +Right keys [1]: [ca_county#71] Join condition: None (87) Project [codegen id : 41] -Output [3]: [ca_county#59, web_sales#64, web_sales#78] -Input [4]: [ca_county#59, web_sales#64, ca_county#73, web_sales#78] +Output [3]: [ca_county#57, web_sales#62, web_sales#76] +Input [4]: [ca_county#57, web_sales#62, ca_county#71, web_sales#76] (88) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, ws_sold_date_sk#82] +Output [3]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, ws_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#82), dynamicpruningexpression(ws_sold_date_sk#82 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (89) ColumnarToRow [codegen id : 35] -Input [3]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, ws_sold_date_sk#82] +Input [3]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, ws_sold_date_sk#80] (90) Filter [codegen id : 35] -Input [3]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, ws_sold_date_sk#82] -Condition : isnotnull(ws_bill_addr_sk#80) +Input [3]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, ws_sold_date_sk#80] +Condition : isnotnull(ws_bill_addr_sk#78) (91) ReusedExchange [Reuses operator id: 118] -Output [3]: [d_date_sk#83, d_year#84, d_qoy#85] +Output [3]: [d_date_sk#81, d_year#82, d_qoy#83] (92) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_sold_date_sk#82] -Right keys [1]: [d_date_sk#83] +Left keys [1]: [ws_sold_date_sk#80] +Right keys [1]: [d_date_sk#81] Join condition: None (93) Project [codegen id : 35] -Output [4]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, d_year#84, d_qoy#85] -Input [6]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, ws_sold_date_sk#82, d_date_sk#83, d_year#84, d_qoy#85] +Output [4]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, d_year#82, d_qoy#83] +Input [6]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, ws_sold_date_sk#80, d_date_sk#81, d_year#82, d_qoy#83] (94) Exchange -Input [4]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, d_year#84, d_qoy#85] -Arguments: hashpartitioning(ws_bill_addr_sk#80, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [4]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, d_year#82, d_qoy#83] +Arguments: hashpartitioning(ws_bill_addr_sk#78, 5), ENSURE_REQUIREMENTS, [id=#84] (95) Sort [codegen id : 36] -Input [4]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, d_year#84, d_qoy#85] -Arguments: [ws_bill_addr_sk#80 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, d_year#82, d_qoy#83] +Arguments: [ws_bill_addr_sk#78 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#87, ca_county#88] +Output [2]: [ca_address_sk#85, ca_county#86] (97) Sort [codegen id : 38] -Input [2]: [ca_address_sk#87, ca_county#88] -Arguments: [ca_address_sk#87 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#85, ca_county#86] +Arguments: [ca_address_sk#85 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 39] -Left keys [1]: [ws_bill_addr_sk#80] -Right keys [1]: [ca_address_sk#87] +Left keys [1]: [ws_bill_addr_sk#78] +Right keys [1]: [ca_address_sk#85] Join condition: None (99) Project [codegen id : 39] -Output [4]: [ws_ext_sales_price#81, d_year#84, d_qoy#85, ca_county#88] -Input [6]: [ws_bill_addr_sk#80, ws_ext_sales_price#81, d_year#84, d_qoy#85, ca_address_sk#87, ca_county#88] +Output [4]: [ws_ext_sales_price#79, d_year#82, d_qoy#83, ca_county#86] +Input [6]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, d_year#82, d_qoy#83, ca_address_sk#85, ca_county#86] (100) HashAggregate [codegen id : 39] -Input [4]: [ws_ext_sales_price#81, d_year#84, d_qoy#85, ca_county#88] -Keys [3]: [ca_county#88, d_qoy#85, d_year#84] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#81))] -Aggregate Attributes [1]: [sum#89] -Results [4]: [ca_county#88, d_qoy#85, d_year#84, sum#90] +Input [4]: [ws_ext_sales_price#79, d_year#82, d_qoy#83, ca_county#86] +Keys [3]: [ca_county#86, d_qoy#83, d_year#82] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#79))] +Aggregate Attributes [1]: [sum#87] +Results [4]: [ca_county#86, d_qoy#83, d_year#82, sum#88] (101) Exchange -Input [4]: [ca_county#88, d_qoy#85, d_year#84, sum#90] -Arguments: hashpartitioning(ca_county#88, d_qoy#85, d_year#84, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [4]: [ca_county#86, d_qoy#83, d_year#82, sum#88] +Arguments: hashpartitioning(ca_county#86, d_qoy#83, d_year#82, 5), ENSURE_REQUIREMENTS, [id=#89] (102) HashAggregate [codegen id : 40] -Input [4]: [ca_county#88, d_qoy#85, d_year#84, sum#90] -Keys [3]: [ca_county#88, d_qoy#85, d_year#84] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#81))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#81))#92] -Results [2]: [ca_county#88, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#81))#92,17,2) AS web_sales#93] +Input [4]: [ca_county#86, d_qoy#83, d_year#82, sum#88] +Keys [3]: [ca_county#86, d_qoy#83, d_year#82] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#79))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#79))#90] +Results [2]: [ca_county#86, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#90,17,2) AS web_sales#91] (103) BroadcastExchange -Input [2]: [ca_county#88, web_sales#93] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#94] +Input [2]: [ca_county#86, web_sales#91] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#92] (104) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#59] -Right keys [1]: [ca_county#88] +Left keys [1]: [ca_county#57] +Right keys [1]: [ca_county#86] Join condition: None (105) Project [codegen id : 41] -Output [6]: [ca_county#59, web_sales#64, web_sales#78, web_sales#93, CASE WHEN (web_sales#64 > 0.00) THEN CheckOverflow((promote_precision(web_sales#78) / promote_precision(web_sales#64)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (ws1.web_sales > 0.00BD) THEN (ws2.web_sales / ws1.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#95, CASE WHEN (web_sales#78 > 0.00) THEN CheckOverflow((promote_precision(web_sales#93) / promote_precision(web_sales#78)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (ws2.web_sales > 0.00BD) THEN (ws3.web_sales / ws2.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#96] -Input [5]: [ca_county#59, web_sales#64, web_sales#78, ca_county#88, web_sales#93] +Output [4]: [ca_county#57, web_sales#62, web_sales#76, web_sales#91] +Input [5]: [ca_county#57, web_sales#62, web_sales#76, ca_county#86, web_sales#91] (106) BroadcastExchange -Input [6]: [ca_county#59, web_sales#64, web_sales#78, web_sales#93, CASE WHEN (ws1.web_sales > 0.00BD) THEN (ws2.web_sales / ws1.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#95, CASE WHEN (ws2.web_sales > 0.00BD) THEN (ws3.web_sales / ws2.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#96] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#97] +Input [4]: [ca_county#57, web_sales#62, web_sales#76, web_sales#91] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#93] (107) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#42] -Right keys [1]: [ca_county#59] -Join condition: ((CASE WHEN (ws1.web_sales > 0.00BD) THEN (ws2.web_sales / ws1.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#95 > CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#49) AND (CASE WHEN (ws2.web_sales > 0.00BD) THEN (ws3.web_sales / ws2.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#96 > CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#50)) +Right keys [1]: [ca_county#57] +Join condition: ((CASE WHEN (web_sales#62 > 0.00) THEN CheckOverflow((promote_precision(web_sales#76) / promote_precision(web_sales#62)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#47 > 0.00) THEN CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#47)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#76 > 0.00) THEN CheckOverflow((promote_precision(web_sales#91) / promote_precision(web_sales#76)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#16 > 0.00) THEN CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) ELSE null END)) (108) Project [codegen id : 42] -Output [6]: [ca_county#42, d_year#38, CheckOverflow((promote_precision(web_sales#78) / promote_precision(web_sales#64)), DecimalType(37,20), true) AS web_q1_q2_increase#98, CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#47)), DecimalType(37,20), true) AS store_q1_q2_increase#99, CheckOverflow((promote_precision(web_sales#93) / promote_precision(web_sales#78)), DecimalType(37,20), true) AS web_q2_q3_increase#100, CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q2_q3_increase#101] -Input [13]: [store_sales#16, store_sales#31, ca_county#42, d_year#38, store_sales#47, CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#49, CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#50, ca_county#59, web_sales#64, web_sales#78, web_sales#93, CASE WHEN (ws1.web_sales > 0.00BD) THEN (ws2.web_sales / ws1.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#95, CASE WHEN (ws2.web_sales > 0.00BD) THEN (ws3.web_sales / ws2.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#96] +Output [6]: [ca_county#42, d_year#38, CheckOverflow((promote_precision(web_sales#76) / promote_precision(web_sales#62)), DecimalType(37,20), true) AS web_q1_q2_increase#94, CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#47)), DecimalType(37,20), true) AS store_q1_q2_increase#95, CheckOverflow((promote_precision(web_sales#91) / promote_precision(web_sales#76)), DecimalType(37,20), true) AS web_q2_q3_increase#96, CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q2_q3_increase#97] +Input [9]: [store_sales#16, store_sales#31, ca_county#42, d_year#38, store_sales#47, ca_county#57, web_sales#62, web_sales#76, web_sales#91] (109) Exchange -Input [6]: [ca_county#42, d_year#38, web_q1_q2_increase#98, store_q1_q2_increase#99, web_q2_q3_increase#100, store_q2_q3_increase#101] -Arguments: rangepartitioning(ca_county#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#102] +Input [6]: [ca_county#42, d_year#38, web_q1_q2_increase#94, store_q1_q2_increase#95, web_q2_q3_increase#96, store_q2_q3_increase#97] +Arguments: rangepartitioning(ca_county#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#98] (110) Sort [codegen id : 43] -Input [6]: [ca_county#42, d_year#38, web_q1_q2_increase#98, store_q1_q2_increase#99, web_q2_q3_increase#100, store_q2_q3_increase#101] +Input [6]: [ca_county#42, d_year#38, web_q1_q2_increase#94, store_q1_q2_increase#95, web_q2_q3_increase#96, store_q2_q3_increase#97] Arguments: [ca_county#42 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -638,7 +638,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A (114) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#19 IN dynamicpruning#20 BroadcastExchange (118) @@ -663,7 +663,7 @@ Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 3) (118) BroadcastExchange Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#104] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#100] Subquery:3 Hosting operator id = 37 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 BroadcastExchange (122) @@ -688,12 +688,12 @@ Condition : ((((isnotnull(d_qoy#39) AND isnotnull(d_year#38)) AND (d_qoy#39 = 1) (122) BroadcastExchange Input [3]: [d_date_sk#37, d_year#38, d_qoy#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#105] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#101] -Subquery:4 Hosting operator id = 55 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#36 +Subquery:4 Hosting operator id = 55 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#36 -Subquery:5 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#67 IN dynamicpruning#4 +Subquery:5 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#4 -Subquery:6 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#82 IN dynamicpruning#20 +Subquery:6 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#20 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt index d4620233dc820..ea20602bf0c97 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt @@ -4,7 +4,7 @@ WholeStageCodegen (43) Exchange [ca_county] #1 WholeStageCodegen (42) Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county,CASE WHEN (ws1.web_sales > 0.00BD) THEN (ws2.web_sales / ws1.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END,CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END,CASE WHEN (ws2.web_sales > 0.00BD) THEN (ws3.web_sales / ws2.web_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END,CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] Project [store_sales,store_sales,ca_county,d_year,store_sales] BroadcastHashJoin [ca_county,ca_county] Project [ca_county,store_sales,store_sales] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt index 2cd869cf5e172..9f758c9c4cf6b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt @@ -1,95 +1,94 @@ == Physical Plan == -* Sort (91) -+- Exchange (90) - +- * Project (89) - +- * BroadcastHashJoin Inner BuildRight (88) - :- * Project (74) - : +- * BroadcastHashJoin Inner BuildRight (73) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * BroadcastHashJoin Inner BuildRight (29) - : : : : :- * HashAggregate (15) - : : : : : +- Exchange (14) - : : : : : +- * HashAggregate (13) - : : : : : +- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Project (6) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet default.store_sales (1) - : : : : : : +- ReusedExchange (4) - : : : : : +- BroadcastExchange (10) - : : : : : +- * Filter (9) - : : : : : +- * ColumnarToRow (8) - : : : : : +- Scan parquet default.customer_address (7) - : : : : +- BroadcastExchange (28) - : : : : +- * HashAggregate (27) - : : : : +- Exchange (26) - : : : : +- * HashAggregate (25) - : : : : +- * Project (24) - : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : :- * Project (21) - : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : :- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.store_sales (16) - : : : : : +- ReusedExchange (19) - : : : : +- ReusedExchange (22) - : : : +- BroadcastExchange (42) - : : : +- * HashAggregate (41) - : : : +- Exchange (40) - : : : +- * HashAggregate (39) - : : : +- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : :- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Filter (32) - : : : : : +- * ColumnarToRow (31) - : : : : : +- Scan parquet default.store_sales (30) - : : : : +- ReusedExchange (33) - : : : +- ReusedExchange (36) - : : +- BroadcastExchange (57) - : : +- * HashAggregate (56) - : : +- Exchange (55) - : : +- * HashAggregate (54) - : : +- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Filter (47) - : : : : +- * ColumnarToRow (46) - : : : : +- Scan parquet default.web_sales (45) - : : : +- ReusedExchange (48) - : : +- ReusedExchange (51) - : +- BroadcastExchange (72) - : +- * HashAggregate (71) - : +- Exchange (70) - : +- * HashAggregate (69) - : +- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * Filter (62) - : : : +- * ColumnarToRow (61) - : : : +- Scan parquet default.web_sales (60) - : : +- ReusedExchange (63) - : +- ReusedExchange (66) - +- BroadcastExchange (87) - +- * HashAggregate (86) - +- Exchange (85) - +- * HashAggregate (84) - +- * Project (83) - +- * BroadcastHashJoin Inner BuildRight (82) - :- * Project (80) - : +- * BroadcastHashJoin Inner BuildRight (79) - : :- * Filter (77) - : : +- * ColumnarToRow (76) - : : +- Scan parquet default.web_sales (75) - : +- ReusedExchange (78) - +- ReusedExchange (81) +* Sort (90) ++- Exchange (89) + +- * Project (88) + +- * BroadcastHashJoin Inner BuildRight (87) + :- * Project (73) + : +- * BroadcastHashJoin Inner BuildRight (72) + : :- * BroadcastHashJoin Inner BuildRight (58) + : : :- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * BroadcastHashJoin Inner BuildRight (29) + : : : : :- * HashAggregate (15) + : : : : : +- Exchange (14) + : : : : : +- * HashAggregate (13) + : : : : : +- * Project (12) + : : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : : :- * Project (6) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : : : :- * Filter (3) + : : : : : : : +- * ColumnarToRow (2) + : : : : : : : +- Scan parquet default.store_sales (1) + : : : : : : +- ReusedExchange (4) + : : : : : +- BroadcastExchange (10) + : : : : : +- * Filter (9) + : : : : : +- * ColumnarToRow (8) + : : : : : +- Scan parquet default.customer_address (7) + : : : : +- BroadcastExchange (28) + : : : : +- * HashAggregate (27) + : : : : +- Exchange (26) + : : : : +- * HashAggregate (25) + : : : : +- * Project (24) + : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : :- * Project (21) + : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : :- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.store_sales (16) + : : : : : +- ReusedExchange (19) + : : : : +- ReusedExchange (22) + : : : +- BroadcastExchange (42) + : : : +- * HashAggregate (41) + : : : +- Exchange (40) + : : : +- * HashAggregate (39) + : : : +- * Project (38) + : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : :- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Filter (32) + : : : : : +- * ColumnarToRow (31) + : : : : : +- Scan parquet default.store_sales (30) + : : : : +- ReusedExchange (33) + : : : +- ReusedExchange (36) + : : +- BroadcastExchange (57) + : : +- * HashAggregate (56) + : : +- Exchange (55) + : : +- * HashAggregate (54) + : : +- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * Filter (47) + : : : : +- * ColumnarToRow (46) + : : : : +- Scan parquet default.web_sales (45) + : : : +- ReusedExchange (48) + : : +- ReusedExchange (51) + : +- BroadcastExchange (71) + : +- * HashAggregate (70) + : +- Exchange (69) + : +- * HashAggregate (68) + : +- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Filter (61) + : : : +- * ColumnarToRow (60) + : : : +- Scan parquet default.web_sales (59) + : : +- ReusedExchange (62) + : +- ReusedExchange (65) + +- BroadcastExchange (86) + +- * HashAggregate (85) + +- Exchange (84) + +- * HashAggregate (83) + +- * Project (82) + +- * BroadcastHashJoin Inner BuildRight (81) + :- * Project (79) + : +- * BroadcastHashJoin Inner BuildRight (78) + : :- * Filter (76) + : : +- * ColumnarToRow (75) + : : +- Scan parquet default.web_sales (74) + : +- ReusedExchange (77) + +- ReusedExchange (80) (1) Scan parquet default.store_sales @@ -107,7 +106,7 @@ Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_addr_sk#1) -(4) ReusedExchange [Reuses operator id: 95] +(4) ReusedExchange [Reuses operator id: 94] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] (5) BroadcastHashJoin [codegen id : 3] @@ -179,7 +178,7 @@ Input [3]: [ss_addr_sk#16, ss_ext_sales_price#17, ss_sold_date_sk#18] Input [3]: [ss_addr_sk#16, ss_ext_sales_price#17, ss_sold_date_sk#18] Condition : isnotnull(ss_addr_sk#16) -(19) ReusedExchange [Reuses operator id: 99] +(19) ReusedExchange [Reuses operator id: 98] Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] (20) BroadcastHashJoin [codegen id : 6] @@ -245,7 +244,7 @@ Input [3]: [ss_addr_sk#31, ss_ext_sales_price#32, ss_sold_date_sk#33] Input [3]: [ss_addr_sk#31, ss_ext_sales_price#32, ss_sold_date_sk#33] Condition : isnotnull(ss_addr_sk#31) -(33) ReusedExchange [Reuses operator id: 103] +(33) ReusedExchange [Reuses operator id: 102] Output [3]: [d_date_sk#35, d_year#36, d_qoy#37] (34) BroadcastHashJoin [codegen id : 10] @@ -315,7 +314,7 @@ Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] Condition : isnotnull(ws_bill_addr_sk#46) -(48) ReusedExchange [Reuses operator id: 95] +(48) ReusedExchange [Reuses operator id: 94] Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] (49) BroadcastHashJoin [codegen id : 14] @@ -366,239 +365,235 @@ Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#53] Join condition: None -(59) Project [codegen id : 24] -Output [8]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, CASE WHEN (store_sales#15 > 0.00) THEN CheckOverflow((promote_precision(store_sales#29) / promote_precision(store_sales#15)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#60] -Input [7]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58] - -(60) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] +(59) Scan parquet default.web_sales +Output [3]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, ws_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#19)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] +(60) ColumnarToRow [codegen id : 18] +Input [3]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, ws_sold_date_sk#62] -(62) Filter [codegen id : 18] -Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_addr_sk#61) +(61) Filter [codegen id : 18] +Input [3]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, ws_sold_date_sk#62] +Condition : isnotnull(ws_bill_addr_sk#60) -(63) ReusedExchange [Reuses operator id: 99] -Output [3]: [d_date_sk#64, d_year#65, d_qoy#66] +(62) ReusedExchange [Reuses operator id: 98] +Output [3]: [d_date_sk#63, d_year#64, d_qoy#65] -(64) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#64] +(63) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_sold_date_sk#62] +Right keys [1]: [d_date_sk#63] Join condition: None -(65) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66] -Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65, d_qoy#66] +(64) Project [codegen id : 18] +Output [4]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, d_year#64, d_qoy#65] +Input [6]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, ws_sold_date_sk#62, d_date_sk#63, d_year#64, d_qoy#65] -(66) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#67, ca_county#68] +(65) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#66, ca_county#67] -(67) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#61] -Right keys [1]: [ca_address_sk#67] +(66) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_bill_addr_sk#60] +Right keys [1]: [ca_address_sk#66] Join condition: None -(68) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] -Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_address_sk#67, ca_county#68] - -(69) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] -Keys [3]: [ca_county#68, d_qoy#66, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#62))] -Aggregate Attributes [1]: [sum#69] -Results [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] - -(70) Exchange -Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] -Arguments: hashpartitioning(ca_county#68, d_qoy#66, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#71] - -(71) HashAggregate [codegen id : 19] -Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] -Keys [3]: [ca_county#68, d_qoy#66, d_year#65] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#62))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#62))#72] -Results [2]: [ca_county#68, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#62))#72,17,2) AS web_sales#73] - -(72) BroadcastExchange -Input [2]: [ca_county#68, web_sales#73] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#74] - -(73) BroadcastHashJoin [codegen id : 24] +(67) Project [codegen id : 18] +Output [4]: [ws_ext_sales_price#61, d_year#64, d_qoy#65, ca_county#67] +Input [6]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, d_year#64, d_qoy#65, ca_address_sk#66, ca_county#67] + +(68) HashAggregate [codegen id : 18] +Input [4]: [ws_ext_sales_price#61, d_year#64, d_qoy#65, ca_county#67] +Keys [3]: [ca_county#67, d_qoy#65, d_year#64] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#61))] +Aggregate Attributes [1]: [sum#68] +Results [4]: [ca_county#67, d_qoy#65, d_year#64, sum#69] + +(69) Exchange +Input [4]: [ca_county#67, d_qoy#65, d_year#64, sum#69] +Arguments: hashpartitioning(ca_county#67, d_qoy#65, d_year#64, 5), ENSURE_REQUIREMENTS, [id=#70] + +(70) HashAggregate [codegen id : 19] +Input [4]: [ca_county#67, d_qoy#65, d_year#64, sum#69] +Keys [3]: [ca_county#67, d_qoy#65, d_year#64] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#61))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#61))#71] +Results [2]: [ca_county#67, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#61))#71,17,2) AS web_sales#72] + +(71) BroadcastExchange +Input [2]: [ca_county#67, web_sales#72] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#73] + +(72) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#53] -Right keys [1]: [ca_county#68] -Join condition: (CASE WHEN (web_sales#58 > 0.00) THEN CheckOverflow((promote_precision(web_sales#73) / promote_precision(web_sales#58)), DecimalType(37,20), true) ELSE null END > CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#60) +Right keys [1]: [ca_county#67] +Join condition: (CASE WHEN (web_sales#58 > 0.00) THEN CheckOverflow((promote_precision(web_sales#72) / promote_precision(web_sales#58)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#15 > 0.00) THEN CheckOverflow((promote_precision(store_sales#29) / promote_precision(store_sales#15)), DecimalType(37,20), true) ELSE null END) -(74) Project [codegen id : 24] -Output [9]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, web_sales#73, CASE WHEN (store_sales#29 > 0.00) THEN CheckOverflow((promote_precision(store_sales#44) / promote_precision(store_sales#29)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#75] -Input [10]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#60, ca_county#68, web_sales#73] +(73) Project [codegen id : 24] +Output [8]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, web_sales#72] +Input [9]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, ca_county#67, web_sales#72] -(75) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#76, ws_ext_sales_price#77, ws_sold_date_sk#78] +(74) Scan parquet default.web_sales +Output [3]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, ws_sold_date_sk#76] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_sold_date_sk#78 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(ws_sold_date_sk#76), dynamicpruningexpression(ws_sold_date_sk#76 IN dynamicpruning#34)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#76, ws_ext_sales_price#77, ws_sold_date_sk#78] +(75) ColumnarToRow [codegen id : 22] +Input [3]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, ws_sold_date_sk#76] -(77) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#76, ws_ext_sales_price#77, ws_sold_date_sk#78] -Condition : isnotnull(ws_bill_addr_sk#76) +(76) Filter [codegen id : 22] +Input [3]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, ws_sold_date_sk#76] +Condition : isnotnull(ws_bill_addr_sk#74) -(78) ReusedExchange [Reuses operator id: 103] -Output [3]: [d_date_sk#79, d_year#80, d_qoy#81] +(77) ReusedExchange [Reuses operator id: 102] +Output [3]: [d_date_sk#77, d_year#78, d_qoy#79] -(79) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#78] -Right keys [1]: [d_date_sk#79] +(78) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_sold_date_sk#76] +Right keys [1]: [d_date_sk#77] Join condition: None -(80) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#76, ws_ext_sales_price#77, d_year#80, d_qoy#81] -Input [6]: [ws_bill_addr_sk#76, ws_ext_sales_price#77, ws_sold_date_sk#78, d_date_sk#79, d_year#80, d_qoy#81] +(79) Project [codegen id : 22] +Output [4]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, d_year#78, d_qoy#79] +Input [6]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, ws_sold_date_sk#76, d_date_sk#77, d_year#78, d_qoy#79] -(81) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#82, ca_county#83] +(80) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#80, ca_county#81] -(82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#76] -Right keys [1]: [ca_address_sk#82] +(81) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_bill_addr_sk#74] +Right keys [1]: [ca_address_sk#80] Join condition: None -(83) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#77, d_year#80, d_qoy#81, ca_county#83] -Input [6]: [ws_bill_addr_sk#76, ws_ext_sales_price#77, d_year#80, d_qoy#81, ca_address_sk#82, ca_county#83] - -(84) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#77, d_year#80, d_qoy#81, ca_county#83] -Keys [3]: [ca_county#83, d_qoy#81, d_year#80] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#77))] -Aggregate Attributes [1]: [sum#84] -Results [4]: [ca_county#83, d_qoy#81, d_year#80, sum#85] - -(85) Exchange -Input [4]: [ca_county#83, d_qoy#81, d_year#80, sum#85] -Arguments: hashpartitioning(ca_county#83, d_qoy#81, d_year#80, 5), ENSURE_REQUIREMENTS, [id=#86] - -(86) HashAggregate [codegen id : 23] -Input [4]: [ca_county#83, d_qoy#81, d_year#80, sum#85] -Keys [3]: [ca_county#83, d_qoy#81, d_year#80] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#77))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#77))#87] -Results [2]: [ca_county#83, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#77))#87,17,2) AS web_sales#88] - -(87) BroadcastExchange -Input [2]: [ca_county#83, web_sales#88] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#89] - -(88) BroadcastHashJoin [codegen id : 24] +(82) Project [codegen id : 22] +Output [4]: [ws_ext_sales_price#75, d_year#78, d_qoy#79, ca_county#81] +Input [6]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, d_year#78, d_qoy#79, ca_address_sk#80, ca_county#81] + +(83) HashAggregate [codegen id : 22] +Input [4]: [ws_ext_sales_price#75, d_year#78, d_qoy#79, ca_county#81] +Keys [3]: [ca_county#81, d_qoy#79, d_year#78] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#75))] +Aggregate Attributes [1]: [sum#82] +Results [4]: [ca_county#81, d_qoy#79, d_year#78, sum#83] + +(84) Exchange +Input [4]: [ca_county#81, d_qoy#79, d_year#78, sum#83] +Arguments: hashpartitioning(ca_county#81, d_qoy#79, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#84] + +(85) HashAggregate [codegen id : 23] +Input [4]: [ca_county#81, d_qoy#79, d_year#78, sum#83] +Keys [3]: [ca_county#81, d_qoy#79, d_year#78] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#75))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#75))#85] +Results [2]: [ca_county#81, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#75))#85,17,2) AS web_sales#86] + +(86) BroadcastExchange +Input [2]: [ca_county#81, web_sales#86] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#87] + +(87) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#53] -Right keys [1]: [ca_county#83] -Join condition: (CASE WHEN (web_sales#73 > 0.00) THEN CheckOverflow((promote_precision(web_sales#88) / promote_precision(web_sales#73)), DecimalType(37,20), true) ELSE null END > CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#75) +Right keys [1]: [ca_county#81] +Join condition: (CASE WHEN (web_sales#72 > 0.00) THEN CheckOverflow((promote_precision(web_sales#86) / promote_precision(web_sales#72)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#29 > 0.00) THEN CheckOverflow((promote_precision(store_sales#44) / promote_precision(store_sales#29)), DecimalType(37,20), true) ELSE null END) -(89) Project [codegen id : 24] -Output [6]: [ca_county#9, d_year#6, CheckOverflow((promote_precision(web_sales#73) / promote_precision(web_sales#58)), DecimalType(37,20), true) AS web_q1_q2_increase#90, CheckOverflow((promote_precision(store_sales#29) / promote_precision(store_sales#15)), DecimalType(37,20), true) AS store_q1_q2_increase#91, CheckOverflow((promote_precision(web_sales#88) / promote_precision(web_sales#73)), DecimalType(37,20), true) AS web_q2_q3_increase#92, CheckOverflow((promote_precision(store_sales#44) / promote_precision(store_sales#29)), DecimalType(37,20), true) AS store_q2_q3_increase#93] -Input [11]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, web_sales#73, CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END#75, ca_county#83, web_sales#88] +(88) Project [codegen id : 24] +Output [6]: [ca_county#9, d_year#6, CheckOverflow((promote_precision(web_sales#72) / promote_precision(web_sales#58)), DecimalType(37,20), true) AS web_q1_q2_increase#88, CheckOverflow((promote_precision(store_sales#29) / promote_precision(store_sales#15)), DecimalType(37,20), true) AS store_q1_q2_increase#89, CheckOverflow((promote_precision(web_sales#86) / promote_precision(web_sales#72)), DecimalType(37,20), true) AS web_q2_q3_increase#90, CheckOverflow((promote_precision(store_sales#44) / promote_precision(store_sales#29)), DecimalType(37,20), true) AS store_q2_q3_increase#91] +Input [10]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, web_sales#72, ca_county#81, web_sales#86] -(90) Exchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#90, store_q1_q2_increase#91, web_q2_q3_increase#92, store_q2_q3_increase#93] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#94] +(89) Exchange +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#88, store_q1_q2_increase#89, web_q2_q3_increase#90, store_q2_q3_increase#91] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#92] -(91) Sort [codegen id : 25] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#90, store_q1_q2_increase#91, web_q2_q3_increase#92, store_q2_q3_increase#93] +(90) Sort [codegen id : 25] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#88, store_q1_q2_increase#89, web_q2_q3_increase#90, store_q2_q3_increase#91] Arguments: [ca_county#9 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (95) -+- * Filter (94) - +- * ColumnarToRow (93) - +- Scan parquet default.date_dim (92) +BroadcastExchange (94) ++- * Filter (93) + +- * ColumnarToRow (92) + +- Scan parquet default.date_dim (91) -(92) Scan parquet default.date_dim +(91) Scan parquet default.date_dim Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(93) ColumnarToRow [codegen id : 1] +(92) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(94) Filter [codegen id : 1] +(93) Filter [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(95) BroadcastExchange +(94) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#93] Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (99) -+- * Filter (98) - +- * ColumnarToRow (97) - +- Scan parquet default.date_dim (96) +BroadcastExchange (98) ++- * Filter (97) + +- * ColumnarToRow (96) + +- Scan parquet default.date_dim (95) -(96) Scan parquet default.date_dim +(95) Scan parquet default.date_dim Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(97) ColumnarToRow [codegen id : 1] +(96) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] -(98) Filter [codegen id : 1] +(97) Filter [codegen id : 1] Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] Condition : ((((isnotnull(d_qoy#22) AND isnotnull(d_year#21)) AND (d_qoy#22 = 2)) AND (d_year#21 = 2000)) AND isnotnull(d_date_sk#20)) -(99) BroadcastExchange +(98) BroadcastExchange Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#94] Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#34 -BroadcastExchange (103) -+- * Filter (102) - +- * ColumnarToRow (101) - +- Scan parquet default.date_dim (100) +BroadcastExchange (102) ++- * Filter (101) + +- * ColumnarToRow (100) + +- Scan parquet default.date_dim (99) -(100) Scan parquet default.date_dim +(99) Scan parquet default.date_dim Output [3]: [d_date_sk#35, d_year#36, d_qoy#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(101) ColumnarToRow [codegen id : 1] +(100) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#35, d_year#36, d_qoy#37] -(102) Filter [codegen id : 1] +(101) Filter [codegen id : 1] Input [3]: [d_date_sk#35, d_year#36, d_qoy#37] Condition : ((((isnotnull(d_qoy#37) AND isnotnull(d_year#36)) AND (d_qoy#37 = 3)) AND (d_year#36 = 2000)) AND isnotnull(d_date_sk#35)) -(103) BroadcastExchange +(102) BroadcastExchange Input [3]: [d_date_sk#35, d_year#36, d_qoy#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] Subquery:4 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#19 +Subquery:5 Hosting operator id = 59 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#19 -Subquery:6 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#78 IN dynamicpruning#34 +Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#76 IN dynamicpruning#34 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index 21869f77f47cc..8e57ed5108baf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -4,77 +4,49 @@ WholeStageCodegen (25) Exchange [ca_county] #1 WholeStageCodegen (24) Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,CASE WHEN (ss2.store_sales > 0.00BD) THEN (ss3.store_sales / ss2.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,CASE WHEN (ss1.store_sales > 0.00BD) THEN (ss2.store_sales / ss1.store_sales) ELSE CAST(NULL AS DECIMAL(37,20)) END] - Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county] + Project [ca_county,d_year,store_sales,store_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county] BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (3) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [d_qoy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [ca_address_sk,ca_county] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #6 - WholeStageCodegen (6) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - Filter [d_qoy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 + Exchange [ca_county,d_qoy,d_year] #2 + WholeStageCodegen (3) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk] + ColumnarToRow InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [d_qoy,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [ca_address_sk,ca_county] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_county] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #5 + WholeStageCodegen (7) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #9 - WholeStageCodegen (10) + Exchange [ca_county,d_qoy,d_year] #6 + WholeStageCodegen (6) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,d_year,d_qoy,ca_county] BroadcastHashJoin [ss_addr_sk,ca_address_sk] @@ -84,38 +56,65 @@ WholeStageCodegen (25) ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #10 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 WholeStageCodegen (1) Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 + ReusedExchange [d_date_sk,d_year,d_qoy] #7 InputAdapter ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #12 - WholeStageCodegen (14) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #9 + WholeStageCodegen (10) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #10 + WholeStageCodegen (1) + Filter [d_qoy,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #10 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #12 + WholeStageCodegen (14) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 InputAdapter BroadcastExchange #13 WholeStageCodegen (19) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt index 70c3d52114c27..1ace9e7f294aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -CollectLimit (28) -+- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildLeft (23) +CollectLimit (27) ++- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * Project (23) + : +- * BroadcastHashJoin Inner BuildLeft (22) : :- BroadcastExchange (18) : : +- * Project (17) : : +- * BroadcastHashJoin Inner BuildLeft (16) @@ -22,11 +22,10 @@ CollectLimit (28) : : : +- * ColumnarToRow (7) : : : +- Scan parquet default.catalog_sales (6) : : +- ReusedExchange (9) - : +- * Project (22) - : +- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet default.catalog_sales (19) - +- ReusedExchange (25) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.catalog_sales (19) + +- ReusedExchange (24) (1) Scan parquet default.item @@ -66,7 +65,7 @@ Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] Condition : isnotnull(cs_item_sk#4) -(9) ReusedExchange [Reuses operator id: 33] +(9) ReusedExchange [Reuses operator id: 32] Output [1]: [d_date_sk#8] (10) BroadcastHashJoin [codegen id : 3] @@ -128,66 +127,62 @@ Input [3]: [cs_item_sk#17, cs_ext_discount_amt#18, cs_sold_date_sk#19] Input [3]: [cs_item_sk#17, cs_ext_discount_amt#18, cs_sold_date_sk#19] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_ext_discount_amt#18)) -(22) Project -Output [3]: [cs_item_sk#17, cs_sold_date_sk#19, cast(cs_ext_discount_amt#18 as decimal(14,7)) AS CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7))#20] -Input [3]: [cs_item_sk#17, cs_ext_discount_amt#18, cs_sold_date_sk#19] - -(23) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] Right keys [1]: [cs_item_sk#17] -Join condition: (CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7))#20 > (1.3 * avg(cs_ext_discount_amt))#15) +Join condition: (cast(cs_ext_discount_amt#18 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#15) -(24) Project [codegen id : 6] +(23) Project [codegen id : 6] Output [1]: [cs_sold_date_sk#19] -Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#17, cs_sold_date_sk#19, CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7))#20] +Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#17, cs_ext_discount_amt#18, cs_sold_date_sk#19] -(25) ReusedExchange [Reuses operator id: 33] -Output [1]: [d_date_sk#21] +(24) ReusedExchange [Reuses operator id: 32] +Output [1]: [d_date_sk#20] -(26) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#21] +Right keys [1]: [d_date_sk#20] Join condition: None -(27) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #22] -Input [2]: [cs_sold_date_sk#19, d_date_sk#21] +(26) Project [codegen id : 6] +Output [1]: [1 AS excess discount amount #21] +Input [2]: [cs_sold_date_sk#19, d_date_sk#20] -(28) CollectLimit -Input [1]: [excess discount amount #22] +(27) CollectLimit +Input [1]: [excess discount amount #21] Arguments: 100 ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = cs_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (33) -+- * Project (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet default.date_dim (29) +BroadcastExchange (32) ++- * Project (31) + +- * Filter (30) + +- * ColumnarToRow (29) + +- Scan parquet default.date_dim (28) -(29) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#23] +(28) Scan parquet default.date_dim +Output [2]: [d_date_sk#8, d_date#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#8, d_date#23] +(29) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#8, d_date#22] -(31) Filter [codegen id : 1] -Input [2]: [d_date_sk#8, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) +(30) Filter [codegen id : 1] +Input [2]: [d_date_sk#8, d_date#22] +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#22 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) -(32) Project [codegen id : 1] +(31) Project [codegen id : 1] Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#23] +Input [2]: [d_date_sk#8, d_date#22] -(33) BroadcastExchange +(32) BroadcastExchange Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] Subquery:2 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt index 573c1fdc50c4d..8ca9bf49029f8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt @@ -3,7 +3,7 @@ CollectLimit Project BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk,CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7)),(1.3 * avg(cs_ext_discount_amt))] + BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] InputAdapter BroadcastExchange #1 WholeStageCodegen (4) @@ -39,11 +39,10 @@ CollectLimit Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #4 - Project [cs_item_sk,cs_sold_date_sk,cs_ext_discount_amt] - Filter [cs_item_sk,cs_ext_discount_amt] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [cs_item_sk,cs_ext_discount_amt] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 0c0b8c71f5ae9..f6c9b9ed7dcef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -71,85 +71,85 @@ Right keys [1]: [i_item_sk#5] Join condition: None (10) Project [codegen id : 6] -Output [3]: [cs_sold_date_sk#3, i_item_sk#5, cast(cs_ext_discount_amt#2 as decimal(14,7)) AS CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7))#8] +Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#9, cs_ext_discount_amt#10, cs_sold_date_sk#11] +Output [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#9, cs_ext_discount_amt#10, cs_sold_date_sk#11] +Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] (13) Filter [codegen id : 3] -Input [3]: [cs_item_sk#9, cs_ext_discount_amt#10, cs_sold_date_sk#11] -Condition : isnotnull(cs_item_sk#9) +Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] +Condition : isnotnull(cs_item_sk#8) (14) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#11] (15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#11] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#10] +Right keys [1]: [d_date_sk#11] Join condition: None (16) Project [codegen id : 3] -Output [2]: [cs_item_sk#9, cs_ext_discount_amt#10] -Input [4]: [cs_item_sk#9, cs_ext_discount_amt#10, cs_sold_date_sk#11, d_date_sk#12] +Output [2]: [cs_item_sk#8, cs_ext_discount_amt#9] +Input [4]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10, d_date_sk#11] (17) HashAggregate [codegen id : 3] -Input [2]: [cs_item_sk#9, cs_ext_discount_amt#10] -Keys [1]: [cs_item_sk#9] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#10))] -Aggregate Attributes [2]: [sum#13, count#14] -Results [3]: [cs_item_sk#9, sum#15, count#16] +Input [2]: [cs_item_sk#8, cs_ext_discount_amt#9] +Keys [1]: [cs_item_sk#8] +Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#9))] +Aggregate Attributes [2]: [sum#12, count#13] +Results [3]: [cs_item_sk#8, sum#14, count#15] (18) Exchange -Input [3]: [cs_item_sk#9, sum#15, count#16] -Arguments: hashpartitioning(cs_item_sk#9, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [3]: [cs_item_sk#8, sum#14, count#15] +Arguments: hashpartitioning(cs_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#16] (19) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#9, sum#15, count#16] -Keys [1]: [cs_item_sk#9] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#10))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#10))#18] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#10))#18 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#19, cs_item_sk#9] +Input [3]: [cs_item_sk#8, sum#14, count#15] +Keys [1]: [cs_item_sk#8] +Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))] +Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))#17] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#9))#17 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#18, cs_item_sk#8] (20) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#19, cs_item_sk#9] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#19) +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#18, cs_item_sk#8] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#18) (21) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#19, cs_item_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#20] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#18, cs_item_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] (22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [cs_item_sk#9] -Join condition: (CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7))#8 > (1.3 * avg(cs_ext_discount_amt))#19) +Right keys [1]: [cs_item_sk#8] +Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#18) (23) Project [codegen id : 6] Output [1]: [cs_sold_date_sk#3] -Input [5]: [cs_sold_date_sk#3, i_item_sk#5, CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7))#8, (1.3 * avg(cs_ext_discount_amt))#19, cs_item_sk#9] +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#18, cs_item_sk#8] (24) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#21] +Output [1]: [d_date_sk#20] (25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#21] +Right keys [1]: [d_date_sk#20] Join condition: None (26) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #22] -Input [2]: [cs_sold_date_sk#3, d_date_sk#21] +Output [1]: [1 AS excess discount amount #21] +Input [2]: [cs_sold_date_sk#3, d_date_sk#20] (27) CollectLimit -Input [1]: [excess discount amount #22] +Input [1]: [excess discount amount #21] Arguments: 100 ===== Subqueries ===== @@ -163,27 +163,27 @@ BroadcastExchange (32) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#21, d_date#23] +Output [2]: [d_date_sk#20, d_date#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#21, d_date#23] +Input [2]: [d_date_sk#20, d_date#22] (30) Filter [codegen id : 1] -Input [2]: [d_date_sk#21, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#21)) +Input [2]: [d_date_sk#20, d_date#22] +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#22 <= 2000-04-26)) AND isnotnull(d_date_sk#20)) (31) Project [codegen id : 1] -Output [1]: [d_date_sk#21] -Input [2]: [d_date_sk#21, d_date#23] +Output [1]: [d_date_sk#20] +Input [2]: [d_date_sk#20, d_date#22] (32) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Input [1]: [d_date_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index fcc9dbc36ce95..aea77be43cf05 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -3,8 +3,8 @@ CollectLimit Project BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk,CAST(spark_catalog.default.catalog_sales.cs_ext_discount_amt AS DECIMAL(14,7)),(1.3 * avg(cs_ext_discount_amt))] - Project [cs_sold_date_sk,i_item_sk,cs_ext_discount_amt] + BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] + Project [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_ext_discount_amt] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt index c2c3f165c3940..d6b8e5110ca95 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt @@ -358,271 +358,271 @@ Right keys [1]: [customer_id#85] Join condition: None (53) Project [codegen id : 24] -Output [10]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, CASE WHEN (year_total#26 > 0.000000) THEN CheckOverflow((promote_precision(year_total#58) / promote_precision(year_total#26)), DecimalType(38,14), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#88] +Output [11]: [customer_id#25, year_total#26, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#58, year_total#86] Input [12]: [customer_id#25, year_total#26, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#58, customer_id#85, year_total#86] (54) Scan parquet default.customer -Output [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] +Output [8]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (55) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] +Input [8]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95] (56) Filter [codegen id : 14] -Input [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] -Condition : (isnotnull(c_customer_sk#89) AND isnotnull(c_customer_id#90)) +Input [8]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95] +Condition : (isnotnull(c_customer_sk#88) AND isnotnull(c_customer_id#89)) (57) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] +Output [6]: [cs_bill_customer_sk#96, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#102), dynamicpruningexpression(cs_sold_date_sk#102 IN dynamicpruning#41)] +PartitionFilters: [isnotnull(cs_sold_date_sk#101), dynamicpruningexpression(cs_sold_date_sk#101 IN dynamicpruning#41)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] +Input [6]: [cs_bill_customer_sk#96, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101] (59) Filter [codegen id : 12] -Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] -Condition : isnotnull(cs_bill_customer_sk#97) +Input [6]: [cs_bill_customer_sk#96, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101] +Condition : isnotnull(cs_bill_customer_sk#96) (60) BroadcastExchange -Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] +Input [6]: [cs_bill_customer_sk#96, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#102] (61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#89] -Right keys [1]: [cs_bill_customer_sk#97] +Left keys [1]: [c_customer_sk#88] +Right keys [1]: [cs_bill_customer_sk#96] Join condition: None (62) Project [codegen id : 14] -Output [12]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] -Input [14]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] +Output [12]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101] +Input [14]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, cs_bill_customer_sk#96, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101] (63) ReusedExchange [Reuses operator id: 118] -Output [2]: [d_date_sk#104, d_year#105] +Output [2]: [d_date_sk#103, d_year#104] (64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#102] -Right keys [1]: [d_date_sk#104] +Left keys [1]: [cs_sold_date_sk#101] +Right keys [1]: [d_date_sk#103] Join condition: None (65) Project [codegen id : 14] -Output [12]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, d_year#105] -Input [14]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102, d_date_sk#104, d_year#105] +Output [12]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, d_year#104] +Input [14]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, cs_sold_date_sk#101, d_date_sk#103, d_year#104] (66) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, d_year#105] -Keys [8]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, d_year#105] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#101 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#100 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#98 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#99 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#106, isEmpty#107] -Results [10]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, d_year#105, sum#108, isEmpty#109] +Input [12]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, cs_ext_discount_amt#97, cs_ext_sales_price#98, cs_ext_wholesale_cost#99, cs_ext_list_price#100, d_year#104] +Keys [8]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, d_year#104] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#100 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#99 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#97 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#98 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#105, isEmpty#106] +Results [10]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, d_year#104, sum#107, isEmpty#108] (67) Exchange -Input [10]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, d_year#105, sum#108, isEmpty#109] -Arguments: hashpartitioning(c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, d_year#105, 5), ENSURE_REQUIREMENTS, [id=#110] +Input [10]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, d_year#104, sum#107, isEmpty#108] +Arguments: hashpartitioning(c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, d_year#104, 5), ENSURE_REQUIREMENTS, [id=#109] (68) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, d_year#105, sum#108, isEmpty#109] -Keys [8]: [c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96, d_year#105] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#101 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#100 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#98 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#99 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#101 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#100 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#98 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#99 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#111] -Results [2]: [c_customer_id#90 AS customer_id#112, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#101 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#100 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#98 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#99 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#111 AS year_total#113] +Input [10]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, d_year#104, sum#107, isEmpty#108] +Keys [8]: [c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95, d_year#104] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#100 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#99 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#97 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#98 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#100 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#99 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#97 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#98 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#110] +Results [2]: [c_customer_id#89 AS customer_id#111, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#100 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#99 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#97 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#98 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#110 AS year_total#112] (69) BroadcastExchange -Input [2]: [customer_id#112, year_total#113] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#114] +Input [2]: [customer_id#111, year_total#112] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#113] (70) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#112] -Join condition: (CASE WHEN (year_total#86 > 0.000000) THEN CheckOverflow((promote_precision(year_total#113) / promote_precision(year_total#86)), DecimalType(38,14), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#88) +Right keys [1]: [customer_id#111] +Join condition: (CASE WHEN (year_total#86 > 0.000000) THEN CheckOverflow((promote_precision(year_total#112) / promote_precision(year_total#86)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#26 > 0.000000) THEN CheckOverflow((promote_precision(year_total#58) / promote_precision(year_total#26)), DecimalType(38,14), true) ELSE null END) (71) Project [codegen id : 24] -Output [10]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, year_total#113] -Input [12]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#88, customer_id#112, year_total#113] +Output [10]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, year_total#112] +Input [13]: [customer_id#25, year_total#26, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#58, year_total#86, customer_id#111, year_total#112] (72) Scan parquet default.customer -Output [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] +Output [8]: [c_customer_sk#114, c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (73) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] +Input [8]: [c_customer_sk#114, c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121] (74) Filter [codegen id : 18] -Input [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] -Condition : (isnotnull(c_customer_sk#115) AND isnotnull(c_customer_id#116)) +Input [8]: [c_customer_sk#114, c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121] +Condition : (isnotnull(c_customer_sk#114) AND isnotnull(c_customer_id#115)) (75) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] +Output [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#128), dynamicpruningexpression(ws_sold_date_sk#128 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(ws_sold_date_sk#127), dynamicpruningexpression(ws_sold_date_sk#127 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] +Input [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] (77) Filter [codegen id : 16] -Input [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] -Condition : isnotnull(ws_bill_customer_sk#123) +Input [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] +Condition : isnotnull(ws_bill_customer_sk#122) (78) BroadcastExchange -Input [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#129] +Input [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#128] (79) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#115] -Right keys [1]: [ws_bill_customer_sk#123] +Left keys [1]: [c_customer_sk#114] +Right keys [1]: [ws_bill_customer_sk#122] Join condition: None (80) Project [codegen id : 18] -Output [12]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] -Input [14]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] +Output [12]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] +Input [14]: [c_customer_sk#114, c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] (81) ReusedExchange [Reuses operator id: 114] -Output [2]: [d_date_sk#130, d_year#131] +Output [2]: [d_date_sk#129, d_year#130] (82) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#128] -Right keys [1]: [d_date_sk#130] +Left keys [1]: [ws_sold_date_sk#127] +Right keys [1]: [d_date_sk#129] Join condition: None (83) Project [codegen id : 18] -Output [12]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, d_year#131] -Input [14]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128, d_date_sk#130, d_year#131] +Output [12]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, d_year#130] +Input [14]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127, d_date_sk#129, d_year#130] (84) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, d_year#131] -Keys [8]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#127 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#126 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#124 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#125 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#132, isEmpty#133] -Results [10]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, sum#134, isEmpty#135] +Input [12]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, d_year#130] +Keys [8]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, d_year#130] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#126 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#125 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#123 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#124 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#131, isEmpty#132] +Results [10]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, d_year#130, sum#133, isEmpty#134] (85) Exchange -Input [10]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, sum#134, isEmpty#135] -Arguments: hashpartitioning(c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, 5), ENSURE_REQUIREMENTS, [id=#136] +Input [10]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, d_year#130, sum#133, isEmpty#134] +Arguments: hashpartitioning(c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, d_year#130, 5), ENSURE_REQUIREMENTS, [id=#135] (86) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, sum#134, isEmpty#135] -Keys [8]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#127 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#126 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#124 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#125 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#127 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#126 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#124 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#125 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#137] -Results [2]: [c_customer_id#116 AS customer_id#138, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#127 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#126 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#124 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#125 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#137 AS year_total#139] +Input [10]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, d_year#130, sum#133, isEmpty#134] +Keys [8]: [c_customer_id#115, c_first_name#116, c_last_name#117, c_preferred_cust_flag#118, c_birth_country#119, c_login#120, c_email_address#121, d_year#130] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#126 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#125 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#123 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#124 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#126 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#125 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#123 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#124 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#136] +Results [2]: [c_customer_id#115 AS customer_id#137, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#126 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#125 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#123 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#124 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#136 AS year_total#138] (87) Filter [codegen id : 19] -Input [2]: [customer_id#138, year_total#139] -Condition : (isnotnull(year_total#139) AND (year_total#139 > 0.000000)) +Input [2]: [customer_id#137, year_total#138] +Condition : (isnotnull(year_total#138) AND (year_total#138 > 0.000000)) (88) Project [codegen id : 19] -Output [2]: [customer_id#138 AS customer_id#140, year_total#139 AS year_total#141] -Input [2]: [customer_id#138, year_total#139] +Output [2]: [customer_id#137 AS customer_id#139, year_total#138 AS year_total#140] +Input [2]: [customer_id#137, year_total#138] (89) BroadcastExchange -Input [2]: [customer_id#140, year_total#141] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#142] +Input [2]: [customer_id#139, year_total#140] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#141] (90) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#140] +Right keys [1]: [customer_id#139] Join condition: None (91) Project [codegen id : 24] -Output [10]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#141, CASE WHEN (year_total#86 > 0.000000) THEN CheckOverflow((promote_precision(year_total#113) / promote_precision(year_total#86)), DecimalType(38,14), true) ELSE null END AS CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#143] -Input [12]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, year_total#113, customer_id#140, year_total#141] +Output [11]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, year_total#112, year_total#140] +Input [12]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, year_total#112, customer_id#139, year_total#140] (92) Scan parquet default.customer -Output [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] +Output [8]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (93) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] +Input [8]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149] (94) Filter [codegen id : 22] -Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] -Condition : (isnotnull(c_customer_sk#144) AND isnotnull(c_customer_id#145)) +Input [8]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149] +Condition : (isnotnull(c_customer_sk#142) AND isnotnull(c_customer_id#143)) (95) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Output [6]: [ws_bill_customer_sk#150, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#157), dynamicpruningexpression(ws_sold_date_sk#157 IN dynamicpruning#41)] +PartitionFilters: [isnotnull(ws_sold_date_sk#155), dynamicpruningexpression(ws_sold_date_sk#155 IN dynamicpruning#41)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (96) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Input [6]: [ws_bill_customer_sk#150, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155] (97) Filter [codegen id : 20] -Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] -Condition : isnotnull(ws_bill_customer_sk#152) +Input [6]: [ws_bill_customer_sk#150, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155] +Condition : isnotnull(ws_bill_customer_sk#150) (98) BroadcastExchange -Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#158] +Input [6]: [ws_bill_customer_sk#150, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#156] (99) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#144] -Right keys [1]: [ws_bill_customer_sk#152] +Left keys [1]: [c_customer_sk#142] +Right keys [1]: [ws_bill_customer_sk#150] Join condition: None (100) Project [codegen id : 22] -Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] -Input [14]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Output [12]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155] +Input [14]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, ws_bill_customer_sk#150, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155] (101) ReusedExchange [Reuses operator id: 118] -Output [2]: [d_date_sk#159, d_year#160] +Output [2]: [d_date_sk#157, d_year#158] (102) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#157] -Right keys [1]: [d_date_sk#159] +Left keys [1]: [ws_sold_date_sk#155] +Right keys [1]: [d_date_sk#157] Join condition: None (103) Project [codegen id : 22] -Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] -Input [14]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157, d_date_sk#159, d_year#160] +Output [12]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, d_year#158] +Input [14]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, ws_sold_date_sk#155, d_date_sk#157, d_year#158] (104) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] -Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#161, isEmpty#162] -Results [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] +Input [12]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, ws_ext_discount_amt#151, ws_ext_sales_price#152, ws_ext_wholesale_cost#153, ws_ext_list_price#154, d_year#158] +Keys [8]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, d_year#158] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#154 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#153 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#151 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#152 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#159, isEmpty#160] +Results [10]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, d_year#158, sum#161, isEmpty#162] (105) Exchange -Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] -Arguments: hashpartitioning(c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, 5), ENSURE_REQUIREMENTS, [id=#165] +Input [10]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, d_year#158, sum#161, isEmpty#162] +Arguments: hashpartitioning(c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, d_year#158, 5), ENSURE_REQUIREMENTS, [id=#163] (106) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] -Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166] -Results [2]: [c_customer_id#145 AS customer_id#167, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166 AS year_total#168] +Input [10]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, d_year#158, sum#161, isEmpty#162] +Keys [8]: [c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149, d_year#158] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#154 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#153 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#151 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#152 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#154 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#153 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#151 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#152 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#164] +Results [2]: [c_customer_id#143 AS customer_id#165, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#154 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#153 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#151 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#152 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#164 AS year_total#166] (107) BroadcastExchange -Input [2]: [customer_id#167, year_total#168] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#169] +Input [2]: [customer_id#165, year_total#166] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#167] (108) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#167] -Join condition: (CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#143 > CASE WHEN (year_total#141 > 0.000000) THEN CheckOverflow((promote_precision(year_total#168) / promote_precision(year_total#141)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#165] +Join condition: (CASE WHEN (year_total#86 > 0.000000) THEN CheckOverflow((promote_precision(year_total#112) / promote_precision(year_total#86)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#140 > 0.000000) THEN CheckOverflow((promote_precision(year_total#166) / promote_precision(year_total#140)), DecimalType(38,14), true) ELSE null END) (109) Project [codegen id : 24] Output [7]: [customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57] -Input [12]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#141, CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#143, customer_id#167, year_total#168] +Input [13]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#86, year_total#112, year_total#140, customer_id#165, year_total#166] (110) TakeOrderedAndProject Input [7]: [customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57] @@ -653,7 +653,7 @@ Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_ (114) BroadcastExchange Input [2]: [d_date_sk#17, d_year#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#170] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#168] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#40 IN dynamicpruning#41 BroadcastExchange (118) @@ -678,14 +678,14 @@ Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_ (118) BroadcastExchange Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#171] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#169] Subquery:3 Hosting operator id = 37 Hosting Expression = cs_sold_date_sk#73 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 57 Hosting Expression = cs_sold_date_sk#102 IN dynamicpruning#41 +Subquery:4 Hosting operator id = 57 Hosting Expression = cs_sold_date_sk#101 IN dynamicpruning#41 -Subquery:5 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#128 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#127 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#157 IN dynamicpruning#41 +Subquery:6 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#155 IN dynamicpruning#41 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index e267c917700c2..16a3e25049bcc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -1,12 +1,12 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] WholeStageCodegen (24) Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - BroadcastHashJoin [customer_id,customer_id,CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END] - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] 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 6b24837f0cd90..1617f96c6c4d1 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 @@ -196,53 +196,53 @@ Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_yea Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#26, i_brand#27, s_store_name#28, s_company_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, (rn#37 + 1) AS (v1_lag.rn + 1)#38] +Output [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] Input [8]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] (35) BroadcastExchange -Input [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, (v1_lag.rn + 1)#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], input[5, int, false]),false), [id=#39] +Input [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [id=#38] (36) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#24] -Right keys [5]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, (v1_lag.rn + 1)#38] +Right keys [5]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, (rn#37 + 1)] Join condition: None (37) Project [codegen id : 22] Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [15]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Input [15]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] (38) ReusedExchange [Reuses operator id: 31] -Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] +Output [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] (39) Sort [codegen id : 20] -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, s_store_name#41 ASC NULLS FIRST, s_company_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 (40) Window -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] -Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#40, i_brand#41, s_store_name#42, s_company_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] +Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#39, i_brand#40, s_store_name#41, s_company_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, (rn#47 - 1) AS (v1_lead.rn - 1)#48] -Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] +Output [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] +Input [8]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] (42) BroadcastExchange -Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, (v1_lead.rn - 1)#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], input[5, int, false]),false), [id=#49] +Input [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [id=#47] (43) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#24] -Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (v1_lead.rn - 1)#48] +Right keys [5]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, (rn#46 - 1)] Join condition: None (44) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#50, sum_sales#46 AS nsum#51] -Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, (v1_lead.rn - 1)#48] +Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#48, sum_sales#45 AS nsum#49] +Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] (45) TakeOrderedAndProject -Input [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#50, nsum#51] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#50, nsum#51] +Input [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#48, nsum#49] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#48, nsum#49] ===== Subqueries ===== @@ -269,6 +269,6 @@ Condition : ((((d_year#11 = 1999) OR ((d_year#11 = 1998) AND (d_moy#12 = 12))) O (49) BroadcastExchange Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index de328d5c2904d..4e91752778013 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] WholeStageCodegen (22) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] Filter [avg_monthly_sales,sum_sales] InputAdapter 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 e1fd463439072..83ad4872f96c4 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 @@ -196,53 +196,53 @@ Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sale Arguments: [rank(d_year#28, d_moy#29) windowspecdefinition(i_category#25, i_brand#26, cc_name#27, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#25, i_brand#26, cc_name#27], [d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, (rn#35 + 1) AS (v1_lag.rn + 1)#36] +Output [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] Input [7]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#33, rn#35] (35) BroadcastExchange -Input [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, (v1_lag.rn + 1)#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[4, int, false]),false), [id=#37] +Input [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#36] (36) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#23] -Right keys [4]: [i_category#25, i_brand#26, cc_name#27, (v1_lag.rn + 1)#36] +Right keys [4]: [i_category#25, i_brand#26, cc_name#27, (rn#35 + 1)] Join condition: None (37) Project [codegen id : 22] Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33] -Input [13]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, i_category#25, i_brand#26, cc_name#27, sum_sales#33, (v1_lag.rn + 1)#36] +Input [13]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] (38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +Output [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] (39) Sort [codegen id : 20] -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] +Arguments: [i_category#37 ASC NULLS FIRST, i_brand#38 ASC NULLS FIRST, cc_name#39 ASC NULLS FIRST, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST], false, 0 (40) Window -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -Arguments: [rank(d_year#41, d_moy#42) windowspecdefinition(i_category#38, i_brand#39, cc_name#40, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] +Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] +Arguments: [rank(d_year#40, d_moy#41) windowspecdefinition(i_category#37, i_brand#38, cc_name#39, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#43], [i_category#37, i_brand#38, cc_name#39], [d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, (rn#44 - 1) AS (v1_lead.rn - 1)#45] -Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] +Output [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] +Input [7]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42, rn#43] (42) BroadcastExchange -Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, (v1_lead.rn - 1)#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[4, int, false]),false), [id=#46] +Input [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#44] (43) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#23] -Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (v1_lead.rn - 1)#45] +Right keys [4]: [i_category#37, i_brand#38, cc_name#39, (rn#43 - 1)] Join condition: None (44) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, sum_sales#33 AS psum#47, sum_sales#43 AS nsum#48] -Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33, i_category#38, i_brand#39, cc_name#40, sum_sales#43, (v1_lead.rn - 1)#45] +Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, sum_sales#33 AS psum#45, sum_sales#42 AS nsum#46] +Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33, i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] (45) TakeOrderedAndProject -Input [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#47, nsum#48] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#47, nsum#48] +Input [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#45, nsum#46] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#45, nsum#46] ===== Subqueries ===== @@ -269,6 +269,6 @@ Condition : ((((d_year#11 = 1999) OR ((d_year#11 = 1998) AND (d_moy#12 = 12))) O (49) BroadcastExchange Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index c528a27f75d55..ad877cc7b61e0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] WholeStageCodegen (22) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lead.rn - 1)] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lag.rn + 1)] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] Filter [avg_monthly_sales,sum_sales] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index 05422c6feb00f..d6b61f63b41a8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -1,67 +1,63 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * Project (62) - +- * SortMergeJoin Inner (61) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (28) - : : +- * Project (27) - : : +- * Filter (26) - : : +- * HashAggregate (25) - : : +- Exchange (24) - : : +- * HashAggregate (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (12) - : : : :- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (11) - : : : +- * Project (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.date_dim (7) - : : +- BroadcastExchange (20) - : : +- * Filter (19) - : : +- * ColumnarToRow (18) - : : +- Scan parquet default.item (17) - : +- * Sort (43) - : +- * Project (42) - : +- * Filter (41) - : +- * HashAggregate (40) - : +- Exchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Filter (31) - : : : +- * ColumnarToRow (30) - : : : +- Scan parquet default.catalog_sales (29) - : : +- ReusedExchange (32) - : +- ReusedExchange (35) - +- * Sort (60) - +- * Project (59) - +- * Filter (58) - +- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet default.web_sales (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) +TakeOrderedAndProject (59) ++- * Project (58) + +- * BroadcastHashJoin Inner BuildRight (57) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Filter (26) + : : +- * HashAggregate (25) + : : +- Exchange (24) + : : +- * HashAggregate (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Project (16) + : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (12) + : : : :- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.date_dim (4) + : : : +- BroadcastExchange (11) + : : : +- * Project (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.date_dim (7) + : : +- BroadcastExchange (20) + : : +- * Filter (19) + : : +- * ColumnarToRow (18) + : : +- Scan parquet default.item (17) + : +- BroadcastExchange (40) + : +- * Filter (39) + : +- * HashAggregate (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (32) + : : +- * BroadcastHashJoin Inner BuildRight (31) + : : :- * Filter (29) + : : : +- * ColumnarToRow (28) + : : : +- Scan parquet default.catalog_sales (27) + : : +- ReusedExchange (30) + : +- ReusedExchange (33) + +- BroadcastExchange (56) + +- * Filter (55) + +- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * Project (51) + +- * BroadcastHashJoin Inner BuildRight (50) + :- * Project (48) + : +- * BroadcastHashJoin Inner BuildRight (47) + : :- * Filter (45) + : : +- * ColumnarToRow (44) + : : +- Scan parquet default.web_sales (43) + : +- ReusedExchange (46) + +- ReusedExchange (49) (1) Scan parquet default.store_sales @@ -175,210 +171,194 @@ Results [2]: [i_item_id#13, sum#16] Input [2]: [i_item_id#13, sum#16] Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] -(25) HashAggregate [codegen id : 5] +(25) HashAggregate [codegen id : 15] Input [2]: [i_item_id#13, sum#16] Keys [1]: [i_item_id#13] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#18] Results [2]: [i_item_id#13 AS item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#18,17,2) AS ss_item_rev#20] -(26) Filter [codegen id : 5] +(26) Filter [codegen id : 15] Input [2]: [item_id#19, ss_item_rev#20] Condition : isnotnull(ss_item_rev#20) -(27) Project [codegen id : 5] -Output [6]: [item_id#19, ss_item_rev#20, cast(ss_item_rev#20 as decimal(19,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21, cast(ss_item_rev#20 as decimal(20,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22, CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true) AS (0.90BD * ss_items.ss_item_rev)#23, CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true) AS (1.10BD * ss_items.ss_item_rev)#24] -Input [2]: [item_id#19, ss_item_rev#20] - -(28) Sort [codegen id : 5] -Input [6]: [item_id#19, ss_item_rev#20, CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21, CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22, (0.90BD * ss_items.ss_item_rev)#23, (1.10BD * ss_items.ss_item_rev)#24] -Arguments: [item_id#19 ASC NULLS FIRST], false, 0 - -(29) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +(27) Scan parquet default.catalog_sales +Output [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#27), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 9] -Input [3]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +(28) ColumnarToRow [codegen id : 8] +Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -(31) Filter [codegen id : 9] -Input [3]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] -Condition : isnotnull(cs_item_sk#25) +(29) Filter [codegen id : 8] +Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#21) -(32) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#28] +(30) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#24] -(33) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#28] +(31) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None -(34) Project [codegen id : 9] -Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] +(32) Project [codegen id : 8] +Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] -(35) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#29, i_item_id#30] +(33) ReusedExchange [Reuses operator id: 20] +Output [2]: [i_item_sk#25, i_item_id#26] -(36) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#29] +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#21] +Right keys [1]: [i_item_sk#25] Join condition: None -(37) Project [codegen id : 9] -Output [2]: [cs_ext_sales_price#26, i_item_id#30] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#29, i_item_id#30] +(35) Project [codegen id : 8] +Output [2]: [cs_ext_sales_price#22, i_item_id#26] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#25, i_item_id#26] + +(36) HashAggregate [codegen id : 8] +Input [2]: [cs_ext_sales_price#22, i_item_id#26] +Keys [1]: [i_item_id#26] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] +Aggregate Attributes [1]: [sum#27] +Results [2]: [i_item_id#26, sum#28] + +(37) Exchange +Input [2]: [i_item_id#26, sum#28] +Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [id=#29] (38) HashAggregate [codegen id : 9] -Input [2]: [cs_ext_sales_price#26, i_item_id#30] -Keys [1]: [i_item_id#30] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#31] -Results [2]: [i_item_id#30, sum#32] - -(39) Exchange -Input [2]: [i_item_id#30, sum#32] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] - -(40) HashAggregate [codegen id : 10] -Input [2]: [i_item_id#30, sum#32] -Keys [1]: [i_item_id#30] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#34] -Results [2]: [i_item_id#30 AS item_id#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#34,17,2) AS cs_item_rev#36] - -(41) Filter [codegen id : 10] -Input [2]: [item_id#35, cs_item_rev#36] -Condition : isnotnull(cs_item_rev#36) - -(42) Project [codegen id : 10] -Output [6]: [item_id#35, cs_item_rev#36, CheckOverflow((0.90 * promote_precision(cs_item_rev#36)), DecimalType(19,3), true) AS (0.90BD * cs_items.cs_item_rev)#37, CheckOverflow((1.10 * promote_precision(cs_item_rev#36)), DecimalType(20,3), true) AS (1.10BD * cs_items.cs_item_rev)#38, cast(cs_item_rev#36 as decimal(19,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39, cast(cs_item_rev#36 as decimal(20,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40] -Input [2]: [item_id#35, cs_item_rev#36] - -(43) Sort [codegen id : 10] -Input [6]: [item_id#35, cs_item_rev#36, (0.90BD * cs_items.cs_item_rev)#37, (1.10BD * cs_items.cs_item_rev)#38, CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39, CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40] -Arguments: [item_id#35 ASC NULLS FIRST], false, 0 - -(44) SortMergeJoin [codegen id : 11] +Input [2]: [i_item_id#26, sum#28] +Keys [1]: [i_item_id#26] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] +Results [2]: [i_item_id#26 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] + +(39) Filter [codegen id : 9] +Input [2]: [item_id#31, cs_item_rev#32] +Condition : isnotnull(cs_item_rev#32) + +(40) BroadcastExchange +Input [2]: [item_id#31, cs_item_rev#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] + +(41) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#35] -Join condition: ((((CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21 >= (0.90BD * cs_items.cs_item_rev)#37) AND (CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22 <= (1.10BD * cs_items.cs_item_rev)#38)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39 >= (0.90BD * ss_items.ss_item_rev)#23)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40 <= (1.10BD * ss_items.ss_item_rev)#24)) +Right keys [1]: [item_id#31] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) -(45) Project [codegen id : 11] -Output [11]: [item_id#19, ss_item_rev#20, cs_item_rev#36, cast(ss_item_rev#20 as decimal(19,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#41, cast(ss_item_rev#20 as decimal(20,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#42, cast(cs_item_rev#36 as decimal(19,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#43, cast(cs_item_rev#36 as decimal(20,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#44, CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true) AS (0.90BD * ss_items.ss_item_rev)#45, CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true) AS (1.10BD * ss_items.ss_item_rev)#46, CheckOverflow((0.90 * promote_precision(cs_item_rev#36)), DecimalType(19,3), true) AS (0.90BD * cs_items.cs_item_rev)#47, CheckOverflow((1.10 * promote_precision(cs_item_rev#36)), DecimalType(20,3), true) AS (1.10BD * cs_items.cs_item_rev)#48] -Input [12]: [item_id#19, ss_item_rev#20, CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21, CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22, (0.90BD * ss_items.ss_item_rev)#23, (1.10BD * ss_items.ss_item_rev)#24, item_id#35, cs_item_rev#36, (0.90BD * cs_items.cs_item_rev)#37, (1.10BD * cs_items.cs_item_rev)#38, CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39, CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40] +(42) Project [codegen id : 15] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] +Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] -(46) Scan parquet default.web_sales -Output [3]: [ws_item_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +(43) Scan parquet default.web_sales +Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 15] -Input [3]: [ws_item_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +(44) ColumnarToRow [codegen id : 13] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -(48) Filter [codegen id : 15] -Input [3]: [ws_item_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_item_sk#49) +(45) Filter [codegen id : 13] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#34) -(49) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#52] +(46) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#37] -(50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#52] +(47) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#37] Join condition: None -(51) Project [codegen id : 15] -Output [2]: [ws_item_sk#49, ws_ext_sales_price#50] -Input [4]: [ws_item_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51, d_date_sk#52] +(48) Project [codegen id : 13] +Output [2]: [ws_item_sk#34, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#37] -(52) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#53, i_item_id#54] +(49) ReusedExchange [Reuses operator id: 20] +Output [2]: [i_item_sk#38, i_item_id#39] -(53) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#49] -Right keys [1]: [i_item_sk#53] +(50) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#38] Join condition: None -(54) Project [codegen id : 15] -Output [2]: [ws_ext_sales_price#50, i_item_id#54] -Input [4]: [ws_item_sk#49, ws_ext_sales_price#50, i_item_sk#53, i_item_id#54] - -(55) HashAggregate [codegen id : 15] -Input [2]: [ws_ext_sales_price#50, i_item_id#54] -Keys [1]: [i_item_id#54] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#50))] -Aggregate Attributes [1]: [sum#55] -Results [2]: [i_item_id#54, sum#56] - -(56) Exchange -Input [2]: [i_item_id#54, sum#56] -Arguments: hashpartitioning(i_item_id#54, 5), ENSURE_REQUIREMENTS, [id=#57] - -(57) HashAggregate [codegen id : 16] -Input [2]: [i_item_id#54, sum#56] -Keys [1]: [i_item_id#54] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#50))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#50))#58] -Results [2]: [i_item_id#54 AS item_id#59, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#50))#58,17,2) AS ws_item_rev#60] - -(58) Filter [codegen id : 16] -Input [2]: [item_id#59, ws_item_rev#60] -Condition : isnotnull(ws_item_rev#60) - -(59) Project [codegen id : 16] -Output [6]: [item_id#59, ws_item_rev#60, CheckOverflow((0.90 * promote_precision(ws_item_rev#60)), DecimalType(19,3), true) AS (0.90BD * ws_items.ws_item_rev)#61, CheckOverflow((1.10 * promote_precision(ws_item_rev#60)), DecimalType(20,3), true) AS (1.10BD * ws_items.ws_item_rev)#62, cast(ws_item_rev#60 as decimal(19,3)) AS CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#63, cast(ws_item_rev#60 as decimal(20,3)) AS CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#64] -Input [2]: [item_id#59, ws_item_rev#60] - -(60) Sort [codegen id : 16] -Input [6]: [item_id#59, ws_item_rev#60, (0.90BD * ws_items.ws_item_rev)#61, (1.10BD * ws_items.ws_item_rev)#62, CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#63, CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#64] -Arguments: [item_id#59 ASC NULLS FIRST], false, 0 - -(61) SortMergeJoin [codegen id : 17] +(51) Project [codegen id : 13] +Output [2]: [ws_ext_sales_price#35, i_item_id#39] +Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, i_item_sk#38, i_item_id#39] + +(52) HashAggregate [codegen id : 13] +Input [2]: [ws_ext_sales_price#35, i_item_id#39] +Keys [1]: [i_item_id#39] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#40] +Results [2]: [i_item_id#39, sum#41] + +(53) Exchange +Input [2]: [i_item_id#39, sum#41] +Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, [id=#42] + +(54) HashAggregate [codegen id : 14] +Input [2]: [i_item_id#39, sum#41] +Keys [1]: [i_item_id#39] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] +Results [2]: [i_item_id#39 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] + +(55) Filter [codegen id : 14] +Input [2]: [item_id#44, ws_item_rev#45] +Condition : isnotnull(ws_item_rev#45) + +(56) BroadcastExchange +Input [2]: [item_id#44, ws_item_rev#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] + +(57) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#59] -Join condition: ((((((((CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#41 >= (0.90BD * ws_items.ws_item_rev)#61) AND (CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#42 <= (1.10BD * ws_items.ws_item_rev)#62)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#43 >= (0.90BD * ws_items.ws_item_rev)#61)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#44 <= (1.10BD * ws_items.ws_item_rev)#62)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#63 >= (0.90BD * ss_items.ss_item_rev)#45)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#64 <= (1.10BD * ss_items.ss_item_rev)#46)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#63 >= (0.90BD * cs_items.cs_item_rev)#47)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#64 <= (1.10BD * cs_items.cs_item_rev)#48)) +Right keys [1]: [item_id#44] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) -(62) Project [codegen id : 17] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#60 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#65, cs_item_rev#36, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#36 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#60 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#66, ws_item_rev#60, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#60 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#60 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#67, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#60 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#68] -Input [17]: [item_id#19, ss_item_rev#20, cs_item_rev#36, CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#41, CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#42, CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#43, CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#44, (0.90BD * ss_items.ss_item_rev)#45, (1.10BD * ss_items.ss_item_rev)#46, (0.90BD * cs_items.cs_item_rev)#47, (1.10BD * cs_items.cs_item_rev)#48, item_id#59, ws_item_rev#60, (0.90BD * ws_items.ws_item_rev)#61, (1.10BD * ws_items.ws_item_rev)#62, CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#63, CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#64] +(58) Project [codegen id : 15] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] -(63) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#65, cs_item_rev#36, cs_dev#66, ws_item_rev#60, ws_dev#67, average#68] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#65, cs_item_rev#36, cs_dev#66, ws_item_rev#60, ws_dev#67, average#68] +(59) TakeOrderedAndProject +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* Project (67) -+- * Filter (66) - +- * ColumnarToRow (65) - +- Scan parquet default.date_dim (64) +* Project (63) ++- * Filter (62) + +- * ColumnarToRow (61) + +- Scan parquet default.date_dim (60) -(64) Scan parquet default.date_dim -Output [2]: [d_date#69, d_week_seq#70] +(60) Scan parquet default.date_dim +Output [2]: [d_date#51, d_week_seq#52] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#69, d_week_seq#70] +(61) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#51, d_week_seq#52] -(66) Filter [codegen id : 1] -Input [2]: [d_date#69, d_week_seq#70] -Condition : (isnotnull(d_date#69) AND (d_date#69 = 2000-01-03)) +(62) Filter [codegen id : 1] +Input [2]: [d_date#51, d_week_seq#52] +Condition : (isnotnull(d_date#51) AND (d_date#51 = 2000-01-03)) -(67) Project [codegen id : 1] -Output [1]: [d_week_seq#70] -Input [2]: [d_date#69, d_week_seq#70] +(63) Project [codegen id : 1] +Output [1]: [d_week_seq#52] +Input [2]: [d_date#51, d_week_seq#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt index 52eafb3dfdfa1..57c79b9aee840 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt @@ -1,101 +1,93 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - WholeStageCodegen (17) + WholeStageCodegen (15) Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] - SortMergeJoin [item_id,item_id,CAST(ss_items.ss_item_rev AS DECIMAL(19,3)),(0.90BD * ws_items.ws_item_rev),CAST(ss_items.ss_item_rev AS DECIMAL(20,3)),(1.10BD * ws_items.ws_item_rev),CAST(cs_items.cs_item_rev AS DECIMAL(19,3)),CAST(cs_items.cs_item_rev AS DECIMAL(20,3)),CAST(ws_items.ws_item_rev AS DECIMAL(19,3)),(0.90BD * ss_items.ss_item_rev),CAST(ws_items.ws_item_rev AS DECIMAL(20,3)),(1.10BD * ss_items.ss_item_rev),(0.90BD * cs_items.cs_item_rev),(1.10BD * cs_items.cs_item_rev)] - InputAdapter - WholeStageCodegen (11) - Project [item_id,ss_item_rev,cs_item_rev] - SortMergeJoin [item_id,item_id,CAST(ss_items.ss_item_rev AS DECIMAL(19,3)),(0.90BD * cs_items.cs_item_rev),CAST(ss_items.ss_item_rev AS DECIMAL(20,3)),(1.10BD * cs_items.cs_item_rev),CAST(cs_items.cs_item_rev AS DECIMAL(19,3)),(0.90BD * ss_items.ss_item_rev),CAST(cs_items.cs_item_rev AS DECIMAL(20,3)),(1.10BD * ss_items.ss_item_rev)] + BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] + Project [item_id,ss_item_rev,cs_item_rev] + BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] + Filter [ss_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] InputAdapter - WholeStageCodegen (5) - Sort [item_id] - Project [item_id,ss_item_rev] - Filter [ss_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Exchange [i_item_id] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + Filter [d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date] + Filter [d_week_seq] + Subquery #1 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] + ColumnarToRow InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date] - Filter [d_week_seq] - Subquery #1 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [i_item_sk,i_item_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - WholeStageCodegen (10) - Sort [item_id] - Project [item_id,cs_item_rev] - Filter [cs_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] + Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter - Exchange [i_item_id] #5 - WholeStageCodegen (9) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 + BroadcastExchange #4 + WholeStageCodegen (3) + Filter [i_item_sk,i_item_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (9) + Filter [cs_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] + InputAdapter + Exchange [i_item_id] #6 + WholeStageCodegen (8) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 InputAdapter - WholeStageCodegen (16) - Sort [item_id] - Project [item_id,ws_item_rev] - Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - InputAdapter - Exchange [i_item_id] #6 - WholeStageCodegen (15) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 + BroadcastExchange #7 + WholeStageCodegen (14) + Filter [ws_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] + InputAdapter + Exchange [i_item_id] #8 + WholeStageCodegen (13) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index 47a8eab8bd685..a8ce33ee99d7b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -1,66 +1,63 @@ == Physical Plan == -TakeOrderedAndProject (62) -+- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (27) - : : +- * Filter (26) - : : +- * HashAggregate (25) - : : +- Exchange (24) - : : +- * HashAggregate (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin LeftSemi BuildRight (18) - : : :- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (17) - : : +- * Project (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.date_dim (13) - : +- BroadcastExchange (42) - : +- * Project (41) - : +- * Filter (40) - : +- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet default.catalog_sales (28) - : : +- ReusedExchange (31) - : +- ReusedExchange (34) - +- BroadcastExchange (59) - +- * Project (58) - +- * Filter (57) - +- * HashAggregate (56) - +- Exchange (55) - +- * HashAggregate (54) - +- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Filter (47) - : : +- * ColumnarToRow (46) - : : +- Scan parquet default.web_sales (45) - : +- ReusedExchange (48) - +- ReusedExchange (51) +TakeOrderedAndProject (59) ++- * Project (58) + +- * BroadcastHashJoin Inner BuildRight (57) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Filter (26) + : : +- * HashAggregate (25) + : : +- Exchange (24) + : : +- * HashAggregate (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.item (4) + : : +- BroadcastExchange (20) + : : +- * Project (19) + : : +- * BroadcastHashJoin LeftSemi BuildRight (18) + : : :- * Filter (12) + : : : +- * ColumnarToRow (11) + : : : +- Scan parquet default.date_dim (10) + : : +- BroadcastExchange (17) + : : +- * Project (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.date_dim (13) + : +- BroadcastExchange (40) + : +- * Filter (39) + : +- * HashAggregate (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (32) + : : +- * BroadcastHashJoin Inner BuildRight (31) + : : :- * Filter (29) + : : : +- * ColumnarToRow (28) + : : : +- Scan parquet default.catalog_sales (27) + : : +- ReusedExchange (30) + : +- ReusedExchange (33) + +- BroadcastExchange (56) + +- * Filter (55) + +- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * Project (51) + +- * BroadcastHashJoin Inner BuildRight (50) + :- * Project (48) + : +- * BroadcastHashJoin Inner BuildRight (47) + : :- * Filter (45) + : : +- * ColumnarToRow (44) + : : +- Scan parquet default.web_sales (43) + : +- ReusedExchange (46) + +- ReusedExchange (49) (1) Scan parquet default.store_sales @@ -185,195 +182,183 @@ Results [2]: [i_item_id#5 AS item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sa Input [2]: [item_id#19, ss_item_rev#20] Condition : isnotnull(ss_item_rev#20) -(27) Project [codegen id : 15] -Output [6]: [item_id#19, ss_item_rev#20, cast(ss_item_rev#20 as decimal(19,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21, cast(ss_item_rev#20 as decimal(20,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22, CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true) AS (0.90BD * ss_items.ss_item_rev)#23, CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true) AS (1.10BD * ss_items.ss_item_rev)#24] -Input [2]: [item_id#19, ss_item_rev#20] - -(28) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +(27) Scan parquet default.catalog_sales +Output [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#27), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +(28) ColumnarToRow [codegen id : 8] +Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -(30) Filter [codegen id : 8] -Input [3]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] -Condition : isnotnull(cs_item_sk#25) +(29) Filter [codegen id : 8] +Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#21) -(31) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#28, i_item_id#29] +(30) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#24, i_item_id#25] -(32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#28] +(31) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#21] +Right keys [1]: [i_item_sk#24] Join condition: None -(33) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#26, cs_sold_date_sk#27, i_item_id#29] -Input [5]: [cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, i_item_sk#28, i_item_id#29] +(32) Project [codegen id : 8] +Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25] +Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#24, i_item_id#25] -(34) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#30] +(33) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#26] -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#30] +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#26] Join condition: None -(36) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#26, i_item_id#29] -Input [4]: [cs_ext_sales_price#26, cs_sold_date_sk#27, i_item_id#29, d_date_sk#30] - -(37) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#26, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#31] -Results [2]: [i_item_id#29, sum#32] - -(38) Exchange -Input [2]: [i_item_id#29, sum#32] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, [id=#33] - -(39) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#29, sum#32] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#34] -Results [2]: [i_item_id#29 AS item_id#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#34,17,2) AS cs_item_rev#36] - -(40) Filter [codegen id : 9] -Input [2]: [item_id#35, cs_item_rev#36] -Condition : isnotnull(cs_item_rev#36) - -(41) Project [codegen id : 9] -Output [6]: [item_id#35, cs_item_rev#36, CheckOverflow((0.90 * promote_precision(cs_item_rev#36)), DecimalType(19,3), true) AS (0.90BD * cs_items.cs_item_rev)#37, CheckOverflow((1.10 * promote_precision(cs_item_rev#36)), DecimalType(20,3), true) AS (1.10BD * cs_items.cs_item_rev)#38, cast(cs_item_rev#36 as decimal(19,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39, cast(cs_item_rev#36 as decimal(20,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40] -Input [2]: [item_id#35, cs_item_rev#36] - -(42) BroadcastExchange -Input [6]: [item_id#35, cs_item_rev#36, (0.90BD * cs_items.cs_item_rev)#37, (1.10BD * cs_items.cs_item_rev)#38, CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39, CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#41] - -(43) BroadcastHashJoin [codegen id : 15] +(35) Project [codegen id : 8] +Output [2]: [cs_ext_sales_price#22, i_item_id#25] +Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25, d_date_sk#26] + +(36) HashAggregate [codegen id : 8] +Input [2]: [cs_ext_sales_price#22, i_item_id#25] +Keys [1]: [i_item_id#25] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] +Aggregate Attributes [1]: [sum#27] +Results [2]: [i_item_id#25, sum#28] + +(37) Exchange +Input [2]: [i_item_id#25, sum#28] +Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#29] + +(38) HashAggregate [codegen id : 9] +Input [2]: [i_item_id#25, sum#28] +Keys [1]: [i_item_id#25] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] +Results [2]: [i_item_id#25 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] + +(39) Filter [codegen id : 9] +Input [2]: [item_id#31, cs_item_rev#32] +Condition : isnotnull(cs_item_rev#32) + +(40) BroadcastExchange +Input [2]: [item_id#31, cs_item_rev#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] + +(41) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#35] -Join condition: ((((CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21 >= (0.90BD * cs_items.cs_item_rev)#37) AND (CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22 <= (1.10BD * cs_items.cs_item_rev)#38)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39 >= (0.90BD * ss_items.ss_item_rev)#23)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40 <= (1.10BD * ss_items.ss_item_rev)#24)) +Right keys [1]: [item_id#31] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) -(44) Project [codegen id : 15] -Output [11]: [item_id#19, ss_item_rev#20, cs_item_rev#36, cast(ss_item_rev#20 as decimal(19,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#42, cast(ss_item_rev#20 as decimal(20,3)) AS CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#43, cast(cs_item_rev#36 as decimal(19,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#44, cast(cs_item_rev#36 as decimal(20,3)) AS CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#45, CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true) AS (0.90BD * ss_items.ss_item_rev)#46, CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true) AS (1.10BD * ss_items.ss_item_rev)#47, CheckOverflow((0.90 * promote_precision(cs_item_rev#36)), DecimalType(19,3), true) AS (0.90BD * cs_items.cs_item_rev)#48, CheckOverflow((1.10 * promote_precision(cs_item_rev#36)), DecimalType(20,3), true) AS (1.10BD * cs_items.cs_item_rev)#49] -Input [12]: [item_id#19, ss_item_rev#20, CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#21, CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#22, (0.90BD * ss_items.ss_item_rev)#23, (1.10BD * ss_items.ss_item_rev)#24, item_id#35, cs_item_rev#36, (0.90BD * cs_items.cs_item_rev)#37, (1.10BD * cs_items.cs_item_rev)#38, CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#39, CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#40] +(42) Project [codegen id : 15] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] +Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] -(45) Scan parquet default.web_sales -Output [3]: [ws_item_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] +(43) Scan parquet default.web_sales +Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] +(44) ColumnarToRow [codegen id : 13] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -(47) Filter [codegen id : 13] -Input [3]: [ws_item_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] -Condition : isnotnull(ws_item_sk#50) +(45) Filter [codegen id : 13] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#34) -(48) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#53, i_item_id#54] +(46) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#37, i_item_id#38] -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#50] -Right keys [1]: [i_item_sk#53] +(47) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#37] Join condition: None -(50) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#51, ws_sold_date_sk#52, i_item_id#54] -Input [5]: [ws_item_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52, i_item_sk#53, i_item_id#54] +(48) Project [codegen id : 13] +Output [3]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#38] +Input [5]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_sk#37, i_item_id#38] -(51) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#55] +(49) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#39] -(52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#55] +(50) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#39] Join condition: None -(53) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#51, i_item_id#54] -Input [4]: [ws_ext_sales_price#51, ws_sold_date_sk#52, i_item_id#54, d_date_sk#55] - -(54) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#51, i_item_id#54] -Keys [1]: [i_item_id#54] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#51))] -Aggregate Attributes [1]: [sum#56] -Results [2]: [i_item_id#54, sum#57] - -(55) Exchange -Input [2]: [i_item_id#54, sum#57] -Arguments: hashpartitioning(i_item_id#54, 5), ENSURE_REQUIREMENTS, [id=#58] - -(56) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#54, sum#57] -Keys [1]: [i_item_id#54] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#51))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#51))#59] -Results [2]: [i_item_id#54 AS item_id#60, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#51))#59,17,2) AS ws_item_rev#61] - -(57) Filter [codegen id : 14] -Input [2]: [item_id#60, ws_item_rev#61] -Condition : isnotnull(ws_item_rev#61) - -(58) Project [codegen id : 14] -Output [6]: [item_id#60, ws_item_rev#61, CheckOverflow((0.90 * promote_precision(ws_item_rev#61)), DecimalType(19,3), true) AS (0.90BD * ws_items.ws_item_rev)#62, CheckOverflow((1.10 * promote_precision(ws_item_rev#61)), DecimalType(20,3), true) AS (1.10BD * ws_items.ws_item_rev)#63, cast(ws_item_rev#61 as decimal(19,3)) AS CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#64, cast(ws_item_rev#61 as decimal(20,3)) AS CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#65] -Input [2]: [item_id#60, ws_item_rev#61] - -(59) BroadcastExchange -Input [6]: [item_id#60, ws_item_rev#61, (0.90BD * ws_items.ws_item_rev)#62, (1.10BD * ws_items.ws_item_rev)#63, CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#64, CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#65] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#66] - -(60) BroadcastHashJoin [codegen id : 15] +(51) Project [codegen id : 13] +Output [2]: [ws_ext_sales_price#35, i_item_id#38] +Input [4]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#38, d_date_sk#39] + +(52) HashAggregate [codegen id : 13] +Input [2]: [ws_ext_sales_price#35, i_item_id#38] +Keys [1]: [i_item_id#38] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#40] +Results [2]: [i_item_id#38, sum#41] + +(53) Exchange +Input [2]: [i_item_id#38, sum#41] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [id=#42] + +(54) HashAggregate [codegen id : 14] +Input [2]: [i_item_id#38, sum#41] +Keys [1]: [i_item_id#38] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] +Results [2]: [i_item_id#38 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] + +(55) Filter [codegen id : 14] +Input [2]: [item_id#44, ws_item_rev#45] +Condition : isnotnull(ws_item_rev#45) + +(56) BroadcastExchange +Input [2]: [item_id#44, ws_item_rev#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] + +(57) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#60] -Join condition: ((((((((CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#42 >= (0.90BD * ws_items.ws_item_rev)#62) AND (CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#43 <= (1.10BD * ws_items.ws_item_rev)#63)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#44 >= (0.90BD * ws_items.ws_item_rev)#62)) AND (CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#45 <= (1.10BD * ws_items.ws_item_rev)#63)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#64 >= (0.90BD * ss_items.ss_item_rev)#46)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#65 <= (1.10BD * ss_items.ss_item_rev)#47)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#64 >= (0.90BD * cs_items.cs_item_rev)#48)) AND (CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#65 <= (1.10BD * cs_items.cs_item_rev)#49)) +Right keys [1]: [item_id#44] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) -(61) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#61 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#67, cs_item_rev#36, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#36 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#61 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#68, ws_item_rev#61, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#61 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#61 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#69, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#36 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#61 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#70] -Input [17]: [item_id#19, ss_item_rev#20, cs_item_rev#36, CAST(ss_items.ss_item_rev AS DECIMAL(19,3))#42, CAST(ss_items.ss_item_rev AS DECIMAL(20,3))#43, CAST(cs_items.cs_item_rev AS DECIMAL(19,3))#44, CAST(cs_items.cs_item_rev AS DECIMAL(20,3))#45, (0.90BD * ss_items.ss_item_rev)#46, (1.10BD * ss_items.ss_item_rev)#47, (0.90BD * cs_items.cs_item_rev)#48, (1.10BD * cs_items.cs_item_rev)#49, item_id#60, ws_item_rev#61, (0.90BD * ws_items.ws_item_rev)#62, (1.10BD * ws_items.ws_item_rev)#63, CAST(ws_items.ws_item_rev AS DECIMAL(19,3))#64, CAST(ws_items.ws_item_rev AS DECIMAL(20,3))#65] +(58) Project [codegen id : 15] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] -(62) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#67, cs_item_rev#36, cs_dev#68, ws_item_rev#61, ws_dev#69, average#70] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#67, cs_item_rev#36, cs_dev#68, ws_item_rev#61, ws_dev#69, average#70] +(59) TakeOrderedAndProject +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (66) -+- * Filter (65) - +- * ColumnarToRow (64) - +- Scan parquet default.date_dim (63) +* Project (63) ++- * Filter (62) + +- * ColumnarToRow (61) + +- Scan parquet default.date_dim (60) -(63) Scan parquet default.date_dim -Output [2]: [d_date#71, d_week_seq#72] +(60) Scan parquet default.date_dim +Output [2]: [d_date#51, d_week_seq#52] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#71, d_week_seq#72] +(61) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#51, d_week_seq#52] -(65) Filter [codegen id : 1] -Input [2]: [d_date#71, d_week_seq#72] -Condition : (isnotnull(d_date#71) AND (d_date#71 = 2000-01-03)) +(62) Filter [codegen id : 1] +Input [2]: [d_date#51, d_week_seq#52] +Condition : (isnotnull(d_date#51) AND (d_date#51 = 2000-01-03)) -(66) Project [codegen id : 1] -Output [1]: [d_week_seq#72] -Input [2]: [d_date#71, d_week_seq#72] +(63) Project [codegen id : 1] +Output [1]: [d_week_seq#52] +Input [2]: [d_date#51, d_week_seq#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 7e934ad382a09..1fb6832ea9b4a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -1,96 +1,93 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] WholeStageCodegen (15) Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] - BroadcastHashJoin [item_id,item_id,CAST(ss_items.ss_item_rev AS DECIMAL(19,3)),(0.90BD * ws_items.ws_item_rev),CAST(ss_items.ss_item_rev AS DECIMAL(20,3)),(1.10BD * ws_items.ws_item_rev),CAST(cs_items.cs_item_rev AS DECIMAL(19,3)),CAST(cs_items.cs_item_rev AS DECIMAL(20,3)),CAST(ws_items.ws_item_rev AS DECIMAL(19,3)),(0.90BD * ss_items.ss_item_rev),CAST(ws_items.ws_item_rev AS DECIMAL(20,3)),(1.10BD * ss_items.ss_item_rev),(0.90BD * cs_items.cs_item_rev),(1.10BD * cs_items.cs_item_rev)] + BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] Project [item_id,ss_item_rev,cs_item_rev] - BroadcastHashJoin [item_id,item_id,CAST(ss_items.ss_item_rev AS DECIMAL(19,3)),(0.90BD * cs_items.cs_item_rev),CAST(ss_items.ss_item_rev AS DECIMAL(20,3)),(1.10BD * cs_items.cs_item_rev),CAST(cs_items.cs_item_rev AS DECIMAL(19,3)),(0.90BD * ss_items.ss_item_rev),CAST(cs_items.cs_item_rev AS DECIMAL(20,3)),(1.10BD * ss_items.ss_item_rev)] - Project [item_id,ss_item_rev] - Filter [ss_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow + BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] + Filter [ss_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + Filter [i_item_sk,i_item_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (3) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + Filter [d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [i_item_sk,i_item_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date] - Filter [d_week_seq] - Subquery #1 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + BroadcastExchange #4 + WholeStageCodegen (2) + Project [d_date] + Filter [d_week_seq] + Subquery #1 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Project [item_id,cs_item_rev] - Filter [cs_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - InputAdapter - Exchange [i_item_id] #6 - WholeStageCodegen (8) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 - InputAdapter - ReusedExchange [d_date_sk] #3 + Filter [cs_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] + InputAdapter + Exchange [i_item_id] #6 + WholeStageCodegen (8) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #2 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #7 WholeStageCodegen (14) - Project [item_id,ws_item_rev] - Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (13) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 - InputAdapter - ReusedExchange [d_date_sk] #3 + Filter [ws_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] + InputAdapter + Exchange [i_item_id] #8 + WholeStageCodegen (13) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #2 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt index c07e395777b7d..cf22f178dd33f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt @@ -228,21 +228,21 @@ Right keys [1]: [d_week_seq#76] Join condition: None (40) Project [codegen id : 9] -Output [9]: [s_store_id#73 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, tue_sales#32 AS tue_sales2#81, wed_sales#33 AS wed_sales2#82, thu_sales#34 AS thu_sales2#83, fri_sales#35 AS fri_sales2#84, sat_sales#36 AS sat_sales2#85, (d_week_seq#54 - 52) AS (x.d_week_seq2 - 52)#86] +Output [9]: [d_week_seq#54 AS d_week_seq2#78, s_store_id#73 AS s_store_id2#79, sun_sales#30 AS sun_sales2#80, mon_sales#31 AS mon_sales2#81, tue_sales#32 AS tue_sales2#82, wed_sales#33 AS wed_sales2#83, thu_sales#34 AS thu_sales2#84, fri_sales#35 AS fri_sales2#85, sat_sales#36 AS sat_sales2#86] Input [10]: [d_week_seq#54, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#73, d_week_seq#76] (41) BroadcastExchange -Input [9]: [s_store_id2#78, sun_sales2#79, mon_sales2#80, tue_sales2#81, wed_sales2#82, thu_sales2#83, fri_sales2#84, sat_sales2#85, (x.d_week_seq2 - 52)#86] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[8, int, true]),false), [id=#87] +Input [9]: [d_week_seq2#78, s_store_id2#79, sun_sales2#80, mon_sales2#81, tue_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#87] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#78, (x.d_week_seq2 - 52)#86] +Right keys [2]: [s_store_id2#79, (d_week_seq2#78 - 52)] Join condition: None (43) Project [codegen id : 10] -Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#88, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#89, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#81)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#90, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#82)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#91, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#83)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#92, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#84)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#93, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#85)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#94] -Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, s_store_id2#78, sun_sales2#79, mon_sales2#80, tue_sales2#81, wed_sales2#82, thu_sales2#83, fri_sales2#84, sat_sales2#85, (x.d_week_seq2 - 52)#86] +Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#80)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#88, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#81)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#89, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#82)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#90, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#83)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#91, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#84)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#92, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#85)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#93, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#86)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#94] +Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#78, s_store_id2#79, sun_sales2#80, mon_sales2#81, tue_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] (44) TakeOrderedAndProject Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#88, (mon_sales1 / mon_sales2)#89, (tue_sales1 / tue_sales2)#90, (wed_sales1 / wed_sales2)#91, (thu_sales1 / thu_sales2)#92, (fri_sales1 / fri_sales2)#93, (sat_sales1 / sat_sales2)#94] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt index 99748e7fd3233..541817641b129 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] WholeStageCodegen (10) Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,(x.d_week_seq2 - 52)] + BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] @@ -42,7 +42,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Project [s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + Project [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt index c07e395777b7d..cf22f178dd33f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt @@ -228,21 +228,21 @@ Right keys [1]: [d_week_seq#76] Join condition: None (40) Project [codegen id : 9] -Output [9]: [s_store_id#73 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, tue_sales#32 AS tue_sales2#81, wed_sales#33 AS wed_sales2#82, thu_sales#34 AS thu_sales2#83, fri_sales#35 AS fri_sales2#84, sat_sales#36 AS sat_sales2#85, (d_week_seq#54 - 52) AS (x.d_week_seq2 - 52)#86] +Output [9]: [d_week_seq#54 AS d_week_seq2#78, s_store_id#73 AS s_store_id2#79, sun_sales#30 AS sun_sales2#80, mon_sales#31 AS mon_sales2#81, tue_sales#32 AS tue_sales2#82, wed_sales#33 AS wed_sales2#83, thu_sales#34 AS thu_sales2#84, fri_sales#35 AS fri_sales2#85, sat_sales#36 AS sat_sales2#86] Input [10]: [d_week_seq#54, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#73, d_week_seq#76] (41) BroadcastExchange -Input [9]: [s_store_id2#78, sun_sales2#79, mon_sales2#80, tue_sales2#81, wed_sales2#82, thu_sales2#83, fri_sales2#84, sat_sales2#85, (x.d_week_seq2 - 52)#86] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[8, int, true]),false), [id=#87] +Input [9]: [d_week_seq2#78, s_store_id2#79, sun_sales2#80, mon_sales2#81, tue_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#87] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#78, (x.d_week_seq2 - 52)#86] +Right keys [2]: [s_store_id2#79, (d_week_seq2#78 - 52)] Join condition: None (43) Project [codegen id : 10] -Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#88, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#89, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#81)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#90, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#82)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#91, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#83)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#92, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#84)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#93, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#85)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#94] -Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, s_store_id2#78, sun_sales2#79, mon_sales2#80, tue_sales2#81, wed_sales2#82, thu_sales2#83, fri_sales2#84, sat_sales2#85, (x.d_week_seq2 - 52)#86] +Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#80)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#88, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#81)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#89, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#82)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#90, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#83)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#91, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#84)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#92, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#85)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#93, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#86)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#94] +Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#78, s_store_id2#79, sun_sales2#80, mon_sales2#81, tue_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] (44) TakeOrderedAndProject Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#88, (mon_sales1 / mon_sales2)#89, (tue_sales1 / tue_sales2)#90, (wed_sales1 / wed_sales2)#91, (thu_sales1 / thu_sales2)#92, (fri_sales1 / fri_sales2)#93, (sat_sales1 / sat_sales2)#94] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt index 99748e7fd3233..541817641b129 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] WholeStageCodegen (10) Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,(x.d_week_seq2 - 52)] + BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] @@ -42,7 +42,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Project [s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + Project [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt index 16850fd1593b6..e4baf3b296016 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt @@ -1,46 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- * SortMergeJoin Inner (40) - :- * Sort (34) - : +- Exchange (33) - : +- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (11) - : : : +- * Filter (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (24) - : : +- * HashAggregate (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- * HashAggregate (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet default.store_sales (12) - : : +- ReusedExchange (15) - : +- BroadcastExchange (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet default.store (27) - +- * Sort (39) - +- Exchange (38) - +- * Filter (37) - +- * ColumnarToRow (36) - +- Scan parquet default.item (35) +TakeOrderedAndProject (41) ++- * Project (40) + +- * SortMergeJoin Inner (39) + :- * Sort (33) + : +- Exchange (32) + : +- * Project (31) + : +- * BroadcastHashJoin Inner BuildRight (30) + : :- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Filter (10) + : : : +- * HashAggregate (9) + : : : +- Exchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (23) + : : +- * HashAggregate (22) + : : +- Exchange (21) + : : +- * HashAggregate (20) + : : +- * HashAggregate (19) + : : +- Exchange (18) + : : +- * HashAggregate (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Filter (13) + : : : +- * ColumnarToRow (12) + : : : +- Scan parquet default.store_sales (11) + : : +- ReusedExchange (14) + : +- BroadcastExchange (29) + : +- * Filter (28) + : +- * ColumnarToRow (27) + : +- Scan parquet default.store (26) + +- * Sort (38) + +- Exchange (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.item (34) (1) Scan parquet default.store_sales @@ -58,7 +57,7 @@ Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4] Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) -(4) ReusedExchange [Reuses operator id: 47] +(4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 2] @@ -92,188 +91,184 @@ Results [3]: [ss_store_sk#2, ss_item_sk#1, MakeDecimal(sum(UnscaledValue(ss_sale Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] Condition : isnotnull(revenue#11) -(11) Project [codegen id : 8] -Output [4]: [ss_store_sk#2, ss_item_sk#1, revenue#11, cast(revenue#11 as decimal(23,7)) AS CAST(sc.revenue AS DECIMAL(23,7))#12] -Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] - -(12) Scan parquet default.store_sales -Output [4]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16] +(11) Scan parquet default.store_sales +Output [4]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_sold_date_sk#16 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 4] -Input [4]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16] +(12) ColumnarToRow [codegen id : 4] +Input [4]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -(14) Filter [codegen id : 4] -Input [4]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16] -Condition : isnotnull(ss_store_sk#14) +(13) Filter [codegen id : 4] +Input [4]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +Condition : isnotnull(ss_store_sk#13) -(15) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#17] +(14) ReusedExchange [Reuses operator id: 46] +Output [1]: [d_date_sk#16] -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#16] -Right keys [1]: [d_date_sk#17] +(15) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#15] +Right keys [1]: [d_date_sk#16] Join condition: None -(17) Project [codegen id : 4] -Output [3]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15] -Input [5]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15, ss_sold_date_sk#16, d_date_sk#17] +(16) Project [codegen id : 4] +Output [3]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14] +Input [5]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#16] + +(17) HashAggregate [codegen id : 4] +Input [3]: [ss_item_sk#12, ss_store_sk#13, ss_sales_price#14] +Keys [2]: [ss_store_sk#13, ss_item_sk#12] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] +Aggregate Attributes [1]: [sum#17] +Results [3]: [ss_store_sk#13, ss_item_sk#12, sum#18] -(18) HashAggregate [codegen id : 4] -Input [3]: [ss_item_sk#13, ss_store_sk#14, ss_sales_price#15] -Keys [2]: [ss_store_sk#14, ss_item_sk#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#15))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [ss_store_sk#14, ss_item_sk#13, sum#19] +(18) Exchange +Input [3]: [ss_store_sk#13, ss_item_sk#12, sum#18] +Arguments: hashpartitioning(ss_store_sk#13, ss_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#19] -(19) Exchange -Input [3]: [ss_store_sk#14, ss_item_sk#13, sum#19] -Arguments: hashpartitioning(ss_store_sk#14, ss_item_sk#13, 5), ENSURE_REQUIREMENTS, [id=#20] +(19) HashAggregate [codegen id : 5] +Input [3]: [ss_store_sk#13, ss_item_sk#12, sum#18] +Keys [2]: [ss_store_sk#13, ss_item_sk#12] +Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] +Results [2]: [ss_store_sk#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS revenue#21] (20) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#14, ss_item_sk#13, sum#19] -Keys [2]: [ss_store_sk#14, ss_item_sk#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#15))#21] -Results [2]: [ss_store_sk#14, MakeDecimal(sum(UnscaledValue(ss_sales_price#15))#21,17,2) AS revenue#22] - -(21) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#14, revenue#22] -Keys [1]: [ss_store_sk#14] -Functions [1]: [partial_avg(revenue#22)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ss_store_sk#14, sum#25, count#26] - -(22) Exchange -Input [3]: [ss_store_sk#14, sum#25, count#26] -Arguments: hashpartitioning(ss_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#27] - -(23) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#14, sum#25, count#26] -Keys [1]: [ss_store_sk#14] -Functions [1]: [avg(revenue#22)] -Aggregate Attributes [1]: [avg(revenue#22)#28] -Results [2]: [ss_store_sk#14, CheckOverflow((0.100000 * promote_precision(avg(revenue#22)#28)), DecimalType(23,7), true) AS (0.100000BD * sb.ave)#29] - -(24) BroadcastExchange -Input [2]: [ss_store_sk#14, (0.100000BD * sb.ave)#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] - -(25) BroadcastHashJoin [codegen id : 8] +Input [2]: [ss_store_sk#13, revenue#21] +Keys [1]: [ss_store_sk#13] +Functions [1]: [partial_avg(revenue#21)] +Aggregate Attributes [2]: [sum#22, count#23] +Results [3]: [ss_store_sk#13, sum#24, count#25] + +(21) Exchange +Input [3]: [ss_store_sk#13, sum#24, count#25] +Arguments: hashpartitioning(ss_store_sk#13, 5), ENSURE_REQUIREMENTS, [id=#26] + +(22) HashAggregate [codegen id : 6] +Input [3]: [ss_store_sk#13, sum#24, count#25] +Keys [1]: [ss_store_sk#13] +Functions [1]: [avg(revenue#21)] +Aggregate Attributes [1]: [avg(revenue#21)#27] +Results [2]: [ss_store_sk#13, avg(revenue#21)#27 AS ave#28] + +(23) BroadcastExchange +Input [2]: [ss_store_sk#13, ave#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] + +(24) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [ss_store_sk#14] -Join condition: (CAST(sc.revenue AS DECIMAL(23,7))#12 <= (0.100000BD * sb.ave)#29) +Right keys [1]: [ss_store_sk#13] +Join condition: (cast(revenue#11 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#28)), DecimalType(23,7), true)) -(26) Project [codegen id : 8] +(25) Project [codegen id : 8] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#11] -Input [6]: [ss_store_sk#2, ss_item_sk#1, revenue#11, CAST(sc.revenue AS DECIMAL(23,7))#12, ss_store_sk#14, (0.100000BD * sb.ave)#29] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#11, ss_store_sk#13, ave#28] -(27) Scan parquet default.store -Output [2]: [s_store_sk#31, s_store_name#32] +(26) Scan parquet default.store +Output [2]: [s_store_sk#30, s_store_name#31] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#31, s_store_name#32] +(27) ColumnarToRow [codegen id : 7] +Input [2]: [s_store_sk#30, s_store_name#31] -(29) Filter [codegen id : 7] -Input [2]: [s_store_sk#31, s_store_name#32] -Condition : isnotnull(s_store_sk#31) +(28) Filter [codegen id : 7] +Input [2]: [s_store_sk#30, s_store_name#31] +Condition : isnotnull(s_store_sk#30) -(30) BroadcastExchange -Input [2]: [s_store_sk#31, s_store_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +(29) BroadcastExchange +Input [2]: [s_store_sk#30, s_store_name#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(31) BroadcastHashJoin [codegen id : 8] +(30) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#31] +Right keys [1]: [s_store_sk#30] Join condition: None -(32) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#11, s_store_name#32] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#11, s_store_sk#31, s_store_name#32] +(31) Project [codegen id : 8] +Output [3]: [ss_item_sk#1, revenue#11, s_store_name#31] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#11, s_store_sk#30, s_store_name#31] -(33) Exchange -Input [3]: [ss_item_sk#1, revenue#11, s_store_name#32] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#34] +(32) Exchange +Input [3]: [ss_item_sk#1, revenue#11, s_store_name#31] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#33] -(34) Sort [codegen id : 9] -Input [3]: [ss_item_sk#1, revenue#11, s_store_name#32] +(33) Sort [codegen id : 9] +Input [3]: [ss_item_sk#1, revenue#11, s_store_name#31] Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 -(35) Scan parquet default.item -Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +(34) Scan parquet default.item +Output [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 10] -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +(35) ColumnarToRow [codegen id : 10] +Input [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] -(37) Filter [codegen id : 10] -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Condition : isnotnull(i_item_sk#35) +(36) Filter [codegen id : 10] +Input [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] +Condition : isnotnull(i_item_sk#34) -(38) Exchange -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: hashpartitioning(i_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#40] +(37) Exchange +Input [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] +Arguments: hashpartitioning(i_item_sk#34, 5), ENSURE_REQUIREMENTS, [id=#39] -(39) Sort [codegen id : 11] -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: [i_item_sk#35 ASC NULLS FIRST], false, 0 +(38) Sort [codegen id : 11] +Input [5]: [i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] +Arguments: [i_item_sk#34 ASC NULLS FIRST], false, 0 -(40) SortMergeJoin [codegen id : 12] +(39) SortMergeJoin [codegen id : 12] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#35] +Right keys [1]: [i_item_sk#34] Join condition: None -(41) Project [codegen id : 12] -Output [6]: [s_store_name#32, i_item_desc#36, revenue#11, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Input [8]: [ss_item_sk#1, revenue#11, s_store_name#32, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +(40) Project [codegen id : 12] +Output [6]: [s_store_name#31, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] +Input [8]: [ss_item_sk#1, revenue#11, s_store_name#31, i_item_sk#34, i_item_desc#35, i_current_price#36, i_wholesale_cost#37, i_brand#38] -(42) TakeOrderedAndProject -Input [6]: [s_store_name#32, i_item_desc#36, revenue#11, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: 100, [s_store_name#32 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#32, i_item_desc#36, revenue#11, i_current_price#37, i_wholesale_cost#38, i_brand#39] +(41) TakeOrderedAndProject +Input [6]: [s_store_name#31, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] +Arguments: 100, [s_store_name#31 ASC NULLS FIRST, i_item_desc#35 ASC NULLS FIRST], [s_store_name#31, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (47) -+- * Project (46) - +- * Filter (45) - +- * ColumnarToRow (44) - +- Scan parquet default.date_dim (43) +BroadcastExchange (46) ++- * Project (45) + +- * Filter (44) + +- * ColumnarToRow (43) + +- Scan parquet default.date_dim (42) -(43) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#41] +(42) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#41] +(43) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#40] -(45) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#41] -Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= 1176)) AND (d_month_seq#41 <= 1187)) AND isnotnull(d_date_sk#6)) +(44) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#40] +Condition : (((isnotnull(d_month_seq#40) AND (d_month_seq#40 >= 1176)) AND (d_month_seq#40 <= 1187)) AND isnotnull(d_date_sk#6)) -(46) Project [codegen id : 1] +(45) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_month_seq#41] +Input [2]: [d_date_sk#6, d_month_seq#40] -(47) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] -Subquery:2 Hosting operator id = 12 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt index 9e6bfde1bf48f..08dbe2abd6959 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt @@ -11,34 +11,33 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes Project [ss_item_sk,revenue,s_store_name] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk,ss_item_sk,revenue] - BroadcastHashJoin [ss_store_sk,ss_store_sk,CAST(sc.revenue AS DECIMAL(23,7)),(0.100000BD * sb.ave)] - Project [ss_store_sk,ss_item_sk,revenue] - Filter [revenue] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - InputAdapter - Exchange [ss_store_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] + Filter [revenue] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] + InputAdapter + Exchange [ss_store_sk,ss_item_sk] #2 + WholeStageCodegen (2) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 WholeStageCodegen (6) - HashAggregate [ss_store_sk,sum,count] [avg(revenue),(0.100000BD * sb.ave),sum,count] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] InputAdapter Exchange [ss_store_sk] #5 WholeStageCodegen (5) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index 7a601935907b0..49cc9f75956a2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -139,84 +139,84 @@ Right keys [1]: [i_item_sk#15] Join condition: None (22) Project [codegen id : 9] -Output [8]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19, cast(revenue#13 as decimal(23,7)) AS CAST(sc.revenue AS DECIMAL(23,7))#21] +Output [7]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19] Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#13, i_item_sk#15, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19] (23) Scan parquet default.store_sales -Output [4]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Output [4]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [4]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Input [4]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24] (25) Filter [codegen id : 6] -Input [4]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_store_sk#23) +Input [4]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24] +Condition : isnotnull(ss_store_sk#22) (26) ReusedExchange [Reuses operator id: 43] -Output [1]: [d_date_sk#26] +Output [1]: [d_date_sk#25] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#26] +Left keys [1]: [ss_sold_date_sk#24] +Right keys [1]: [d_date_sk#25] Join condition: None (28) Project [codegen id : 6] -Output [3]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24] -Input [5]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#26] +Output [3]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23] +Input [5]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23, ss_sold_date_sk#24, d_date_sk#25] (29) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#22, ss_store_sk#23, ss_sales_price#24] -Keys [2]: [ss_store_sk#23, ss_item_sk#22] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] -Aggregate Attributes [1]: [sum#27] -Results [3]: [ss_store_sk#23, ss_item_sk#22, sum#28] +Input [3]: [ss_item_sk#21, ss_store_sk#22, ss_sales_price#23] +Keys [2]: [ss_store_sk#22, ss_item_sk#21] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#23))] +Aggregate Attributes [1]: [sum#26] +Results [3]: [ss_store_sk#22, ss_item_sk#21, sum#27] (30) Exchange -Input [3]: [ss_store_sk#23, ss_item_sk#22, sum#28] -Arguments: hashpartitioning(ss_store_sk#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ss_store_sk#22, ss_item_sk#21, sum#27] +Arguments: hashpartitioning(ss_store_sk#22, ss_item_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] (31) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#23, ss_item_sk#22, sum#28] -Keys [2]: [ss_store_sk#23, ss_item_sk#22] -Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#24))#30] -Results [2]: [ss_store_sk#23, MakeDecimal(sum(UnscaledValue(ss_sales_price#24))#30,17,2) AS revenue#31] +Input [3]: [ss_store_sk#22, ss_item_sk#21, sum#27] +Keys [2]: [ss_store_sk#22, ss_item_sk#21] +Functions [1]: [sum(UnscaledValue(ss_sales_price#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#23))#29] +Results [2]: [ss_store_sk#22, MakeDecimal(sum(UnscaledValue(ss_sales_price#23))#29,17,2) AS revenue#30] (32) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#23, revenue#31] -Keys [1]: [ss_store_sk#23] -Functions [1]: [partial_avg(revenue#31)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ss_store_sk#23, sum#34, count#35] +Input [2]: [ss_store_sk#22, revenue#30] +Keys [1]: [ss_store_sk#22] +Functions [1]: [partial_avg(revenue#30)] +Aggregate Attributes [2]: [sum#31, count#32] +Results [3]: [ss_store_sk#22, sum#33, count#34] (33) Exchange -Input [3]: [ss_store_sk#23, sum#34, count#35] -Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ss_store_sk#22, sum#33, count#34] +Arguments: hashpartitioning(ss_store_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] (34) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#23, sum#34, count#35] -Keys [1]: [ss_store_sk#23] -Functions [1]: [avg(revenue#31)] -Aggregate Attributes [1]: [avg(revenue#31)#37] -Results [2]: [ss_store_sk#23, CheckOverflow((0.100000 * promote_precision(avg(revenue#31)#37)), DecimalType(23,7), true) AS (0.100000BD * sb.ave)#38] +Input [3]: [ss_store_sk#22, sum#33, count#34] +Keys [1]: [ss_store_sk#22] +Functions [1]: [avg(revenue#30)] +Aggregate Attributes [1]: [avg(revenue#30)#36] +Results [2]: [ss_store_sk#22, avg(revenue#30)#36 AS ave#37] (35) BroadcastExchange -Input [2]: [ss_store_sk#23, (0.100000BD * sb.ave)#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [2]: [ss_store_sk#22, ave#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] -Right keys [1]: [ss_store_sk#23] -Join condition: (CAST(sc.revenue AS DECIMAL(23,7))#21 <= (0.100000BD * sb.ave)#38) +Right keys [1]: [ss_store_sk#22] +Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#37)), DecimalType(23,7), true)) (37) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] -Input [10]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19, CAST(sc.revenue AS DECIMAL(23,7))#21, ss_store_sk#23, (0.100000BD * sb.ave)#38] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current_price#17, i_wholesale_cost#18, i_brand#19, ss_store_sk#22, ave#37] (38) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] @@ -233,27 +233,27 @@ BroadcastExchange (43) (39) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#40] +Output [2]: [d_date_sk#8, d_month_seq#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#8, d_month_seq#40] +Input [2]: [d_date_sk#8, d_month_seq#39] (41) Filter [codegen id : 1] -Input [2]: [d_date_sk#8, d_month_seq#40] -Condition : (((isnotnull(d_month_seq#40) AND (d_month_seq#40 >= 1176)) AND (d_month_seq#40 <= 1187)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#8, d_month_seq#39] +Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1176)) AND (d_month_seq#39 <= 1187)) AND isnotnull(d_date_sk#8)) (42) Project [codegen id : 1] Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_month_seq#40] +Input [2]: [d_date_sk#8, d_month_seq#39] (43) BroadcastExchange Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index e407c9cb608b5..4b519f37a58bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] WholeStageCodegen (9) Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_store_sk,ss_store_sk,CAST(sc.revenue AS DECIMAL(23,7)),(0.100000BD * sb.ave)] + BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [s_store_name,ss_store_sk,ss_item_sk,revenue] @@ -45,7 +45,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes InputAdapter BroadcastExchange #5 WholeStageCodegen (8) - HashAggregate [ss_store_sk,sum,count] [avg(revenue),(0.100000BD * sb.ave),sum,count] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] InputAdapter Exchange [ss_store_sk] #6 WholeStageCodegen (7) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt index 6f6786c9eee3f..e838025a71db8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Exchange [cs_item_sk,d_date_sk] #3 WholeStageCodegen (10) Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d1.d_date + INTERVAL '5 days'] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date,i_item_desc] SortMergeJoin [cs_item_sk,i_item_sk] InputAdapter @@ -43,8 +43,25 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (2) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #5 + BroadcastExchange #7 WholeStageCodegen (1) Project [hd_demo_sk] Filter [hd_buy_potential,hd_demo_sk] @@ -52,7 +69,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #6 + BroadcastExchange #8 WholeStageCodegen (2) Project [cd_demo_sk] Filter [cd_marital_status,cd_demo_sk] @@ -60,7 +77,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - BroadcastExchange #7 + BroadcastExchange #9 WholeStageCodegen (3) Filter [d_date,d_date_sk] ColumnarToRow @@ -70,29 +87,14 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom WholeStageCodegen (7) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #8 + Exchange [i_item_sk] #10 WholeStageCodegen (6) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [d_date_sk,d_week_seq,d_date_sk,d_date] - BroadcastHashJoin [d_week_seq,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #5 InputAdapter WholeStageCodegen (14) Sort [inv_item_sk,inv_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt index e684dba8c697c..7968b1cb84729 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d1.d_date + INTERVAL '5 days'] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt index ce30b7c323c39..b6b08c52ef055 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * HashAggregate (49) - +- Exchange (48) - +- * HashAggregate (47) - +- * Project (46) - +- * SortMergeJoin Inner (45) +TakeOrderedAndProject (49) ++- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * Project (45) + +- * SortMergeJoin Inner (44) :- * Sort (14) : +- Exchange (13) : +- * Project (12) @@ -19,36 +19,35 @@ TakeOrderedAndProject (50) : +- * Filter (9) : +- * ColumnarToRow (8) : +- Scan parquet default.store (7) - +- * Sort (44) - +- Exchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin LeftSemi BuildRight (38) - :- * Project (18) - : +- * Filter (17) - : +- * ColumnarToRow (16) - : +- Scan parquet default.customer_address (15) - +- BroadcastExchange (37) - +- * Project (36) - +- * Filter (35) - +- * HashAggregate (34) - +- Exchange (33) - +- * HashAggregate (32) - +- * Project (31) - +- * SortMergeJoin Inner (30) - :- * Sort (23) - : +- Exchange (22) - : +- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet default.customer_address (19) - +- * Sort (29) - +- Exchange (28) - +- * Project (27) - +- * Filter (26) - +- * ColumnarToRow (25) - +- Scan parquet default.customer (24) + +- * Sort (43) + +- Exchange (42) + +- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin LeftSemi BuildRight (37) + :- * Filter (17) + : +- * ColumnarToRow (16) + : +- Scan parquet default.customer_address (15) + +- BroadcastExchange (36) + +- * Project (35) + +- * Filter (34) + +- * HashAggregate (33) + +- Exchange (32) + +- * HashAggregate (31) + +- * Project (30) + +- * SortMergeJoin Inner (29) + :- * Sort (22) + : +- Exchange (21) + : +- * Filter (20) + : +- * ColumnarToRow (19) + : +- Scan parquet default.customer_address (18) + +- * Sort (28) + +- Exchange (27) + +- * Project (26) + +- * Filter (25) + +- * ColumnarToRow (24) + +- Scan parquet default.customer (23) (1) Scan parquet default.store_sales @@ -66,7 +65,7 @@ Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(4) ReusedExchange [Reuses operator id: 55] +(4) ReusedExchange [Reuses operator id: 54] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 3] @@ -126,193 +125,189 @@ Input [1]: [ca_zip#12] Input [1]: [ca_zip#12] Condition : (substr(ca_zip#12, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#12, 1, 5))) -(18) Project [codegen id : 11] -Output [2]: [ca_zip#12, substr(ca_zip#12, 1, 5) AS substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#13] -Input [1]: [ca_zip#12] - -(19) Scan parquet default.customer_address -Output [2]: [ca_address_sk#14, ca_zip#15] +(18) Scan parquet default.customer_address +Output [2]: [ca_address_sk#13, ca_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#14, ca_zip#15] +(19) ColumnarToRow [codegen id : 5] +Input [2]: [ca_address_sk#13, ca_zip#14] -(21) Filter [codegen id : 5] -Input [2]: [ca_address_sk#14, ca_zip#15] -Condition : isnotnull(ca_address_sk#14) +(20) Filter [codegen id : 5] +Input [2]: [ca_address_sk#13, ca_zip#14] +Condition : isnotnull(ca_address_sk#13) -(22) Exchange -Input [2]: [ca_address_sk#14, ca_zip#15] -Arguments: hashpartitioning(ca_address_sk#14, 5), ENSURE_REQUIREMENTS, [id=#16] +(21) Exchange +Input [2]: [ca_address_sk#13, ca_zip#14] +Arguments: hashpartitioning(ca_address_sk#13, 5), ENSURE_REQUIREMENTS, [id=#15] -(23) Sort [codegen id : 6] -Input [2]: [ca_address_sk#14, ca_zip#15] -Arguments: [ca_address_sk#14 ASC NULLS FIRST], false, 0 +(22) Sort [codegen id : 6] +Input [2]: [ca_address_sk#13, ca_zip#14] +Arguments: [ca_address_sk#13 ASC NULLS FIRST], false, 0 -(24) Scan parquet default.customer -Output [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +(23) Scan parquet default.customer +Output [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 7] -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +(24) ColumnarToRow [codegen id : 7] +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] -(26) Filter [codegen id : 7] -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Condition : ((isnotnull(c_preferred_cust_flag#18) AND (c_preferred_cust_flag#18 = Y)) AND isnotnull(c_current_addr_sk#17)) +(25) Filter [codegen id : 7] +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Condition : ((isnotnull(c_preferred_cust_flag#17) AND (c_preferred_cust_flag#17 = Y)) AND isnotnull(c_current_addr_sk#16)) -(27) Project [codegen id : 7] -Output [1]: [c_current_addr_sk#17] -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +(26) Project [codegen id : 7] +Output [1]: [c_current_addr_sk#16] +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] -(28) Exchange -Input [1]: [c_current_addr_sk#17] -Arguments: hashpartitioning(c_current_addr_sk#17, 5), ENSURE_REQUIREMENTS, [id=#19] +(27) Exchange +Input [1]: [c_current_addr_sk#16] +Arguments: hashpartitioning(c_current_addr_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] -(29) Sort [codegen id : 8] -Input [1]: [c_current_addr_sk#17] -Arguments: [c_current_addr_sk#17 ASC NULLS FIRST], false, 0 +(28) Sort [codegen id : 8] +Input [1]: [c_current_addr_sk#16] +Arguments: [c_current_addr_sk#16 ASC NULLS FIRST], false, 0 -(30) SortMergeJoin [codegen id : 9] -Left keys [1]: [ca_address_sk#14] -Right keys [1]: [c_current_addr_sk#17] +(29) SortMergeJoin [codegen id : 9] +Left keys [1]: [ca_address_sk#13] +Right keys [1]: [c_current_addr_sk#16] Join condition: None -(31) Project [codegen id : 9] -Output [1]: [ca_zip#15] -Input [3]: [ca_address_sk#14, ca_zip#15, c_current_addr_sk#17] +(30) Project [codegen id : 9] +Output [1]: [ca_zip#14] +Input [3]: [ca_address_sk#13, ca_zip#14, c_current_addr_sk#16] -(32) HashAggregate [codegen id : 9] -Input [1]: [ca_zip#15] -Keys [1]: [ca_zip#15] +(31) HashAggregate [codegen id : 9] +Input [1]: [ca_zip#14] +Keys [1]: [ca_zip#14] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [2]: [ca_zip#15, count#21] +Aggregate Attributes [1]: [count#19] +Results [2]: [ca_zip#14, count#20] -(33) Exchange -Input [2]: [ca_zip#15, count#21] -Arguments: hashpartitioning(ca_zip#15, 5), ENSURE_REQUIREMENTS, [id=#22] +(32) Exchange +Input [2]: [ca_zip#14, count#20] +Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [id=#21] -(34) HashAggregate [codegen id : 10] -Input [2]: [ca_zip#15, count#21] -Keys [1]: [ca_zip#15] +(33) HashAggregate [codegen id : 10] +Input [2]: [ca_zip#14, count#20] +Keys [1]: [ca_zip#14] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [2]: [substr(ca_zip#15, 1, 5) AS ca_zip#24, count(1)#23 AS cnt#25] +Aggregate Attributes [1]: [count(1)#22] +Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#23, count(1)#22 AS cnt#24] -(35) Filter [codegen id : 10] -Input [2]: [ca_zip#24, cnt#25] -Condition : (cnt#25 > 10) +(34) Filter [codegen id : 10] +Input [2]: [ca_zip#23, cnt#24] +Condition : (cnt#24 > 10) -(36) Project [codegen id : 10] -Output [1]: [ca_zip#24] -Input [2]: [ca_zip#24, cnt#25] +(35) Project [codegen id : 10] +Output [1]: [ca_zip#23] +Input [2]: [ca_zip#23, cnt#24] -(37) BroadcastExchange -Input [1]: [ca_zip#24] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#26] +(36) BroadcastExchange +Input [1]: [ca_zip#23] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#25] -(38) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [coalesce(substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#13, ), isnull(substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#13)] -Right keys [2]: [coalesce(ca_zip#24, ), isnull(ca_zip#24)] +(37) BroadcastHashJoin [codegen id : 11] +Left keys [2]: [coalesce(substr(ca_zip#12, 1, 5), ), isnull(substr(ca_zip#12, 1, 5))] +Right keys [2]: [coalesce(ca_zip#23, ), isnull(ca_zip#23)] Join condition: None -(39) Project [codegen id : 11] -Output [1]: [substr(ca_zip#12, 1, 5) AS ca_zip#27] -Input [2]: [ca_zip#12, substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#13] +(38) Project [codegen id : 11] +Output [1]: [substr(ca_zip#12, 1, 5) AS ca_zip#26] +Input [1]: [ca_zip#12] -(40) HashAggregate [codegen id : 11] -Input [1]: [ca_zip#27] -Keys [1]: [ca_zip#27] +(39) HashAggregate [codegen id : 11] +Input [1]: [ca_zip#26] +Keys [1]: [ca_zip#26] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#27] +Results [1]: [ca_zip#26] -(41) Exchange -Input [1]: [ca_zip#27] -Arguments: hashpartitioning(ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#28] +(40) Exchange +Input [1]: [ca_zip#26] +Arguments: hashpartitioning(ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#27] -(42) HashAggregate [codegen id : 12] -Input [1]: [ca_zip#27] -Keys [1]: [ca_zip#27] +(41) HashAggregate [codegen id : 12] +Input [1]: [ca_zip#26] +Keys [1]: [ca_zip#26] Functions: [] Aggregate Attributes: [] -Results [1]: [substr(ca_zip#27, 1, 2) AS substr(V1.ca_zip, 1, 2)#29] +Results [1]: [substr(ca_zip#26, 1, 2) AS substr(V1.ca_zip, 1, 2)#28] -(43) Exchange -Input [1]: [substr(V1.ca_zip, 1, 2)#29] -Arguments: hashpartitioning(substr(V1.ca_zip, 1, 2)#29, 5), ENSURE_REQUIREMENTS, [id=#30] +(42) Exchange +Input [1]: [substr(V1.ca_zip, 1, 2)#28] +Arguments: hashpartitioning(substr(V1.ca_zip, 1, 2)#28, 5), ENSURE_REQUIREMENTS, [id=#29] -(44) Sort [codegen id : 13] -Input [1]: [substr(V1.ca_zip, 1, 2)#29] -Arguments: [substr(V1.ca_zip, 1, 2)#29 ASC NULLS FIRST], false, 0 +(43) Sort [codegen id : 13] +Input [1]: [substr(V1.ca_zip, 1, 2)#28] +Arguments: [substr(V1.ca_zip, 1, 2)#28 ASC NULLS FIRST], false, 0 -(45) SortMergeJoin [codegen id : 14] +(44) SortMergeJoin [codegen id : 14] Left keys [1]: [substr(spark_catalog.default.store.s_zip, 1, 2)#10] -Right keys [1]: [substr(V1.ca_zip, 1, 2)#29] +Right keys [1]: [substr(V1.ca_zip, 1, 2)#28] Join condition: None -(46) Project [codegen id : 14] +(45) Project [codegen id : 14] Output [2]: [ss_net_profit#2, s_store_name#7] -Input [4]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10, substr(V1.ca_zip, 1, 2)#29] +Input [4]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10, substr(V1.ca_zip, 1, 2)#28] -(47) HashAggregate [codegen id : 14] +(46) HashAggregate [codegen id : 14] Input [2]: [ss_net_profit#2, s_store_name#7] Keys [1]: [s_store_name#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#31] -Results [2]: [s_store_name#7, sum#32] +Aggregate Attributes [1]: [sum#30] +Results [2]: [s_store_name#7, sum#31] -(48) Exchange -Input [2]: [s_store_name#7, sum#32] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#33] +(47) Exchange +Input [2]: [s_store_name#7, sum#31] +Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#32] -(49) HashAggregate [codegen id : 15] -Input [2]: [s_store_name#7, sum#32] +(48) HashAggregate [codegen id : 15] +Input [2]: [s_store_name#7, sum#31] Keys [1]: [s_store_name#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] -Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS sum(ss_net_profit)#35] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#33] +Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#33,17,2) AS sum(ss_net_profit)#34] -(50) TakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#35] -Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#35] +(49) TakeOrderedAndProject +Input [2]: [s_store_name#7, sum(ss_net_profit)#34] +Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (55) -+- * Project (54) - +- * Filter (53) - +- * ColumnarToRow (52) - +- Scan parquet default.date_dim (51) +BroadcastExchange (54) ++- * Project (53) + +- * Filter (52) + +- * ColumnarToRow (51) + +- Scan parquet default.date_dim (50) -(51) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#36, d_qoy#37] +(50) Scan parquet default.date_dim +Output [3]: [d_date_sk#5, d_year#35, d_qoy#36] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#36, d_qoy#37] +(51) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#5, d_year#35, d_qoy#36] -(53) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#36, d_qoy#37] -Condition : ((((isnotnull(d_qoy#37) AND isnotnull(d_year#36)) AND (d_qoy#37 = 2)) AND (d_year#36 = 1998)) AND isnotnull(d_date_sk#5)) +(52) Filter [codegen id : 1] +Input [3]: [d_date_sk#5, d_year#35, d_qoy#36] +Condition : ((((isnotnull(d_qoy#36) AND isnotnull(d_year#35)) AND (d_qoy#36 = 2)) AND (d_year#35 = 1998)) AND isnotnull(d_date_sk#5)) -(54) Project [codegen id : 1] +(53) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#36, d_qoy#37] +Input [3]: [d_date_sk#5, d_year#35, d_qoy#36] -(55) BroadcastExchange +(54) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt index aab384d8e6512..435254f45f10d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt @@ -50,12 +50,11 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (11) HashAggregate [ca_zip] Project [ca_zip] - BroadcastHashJoin [substr(spark_catalog.default.customer_address.ca_zip, 1, 5),ca_zip] - Project [ca_zip] - Filter [ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_zip] + BroadcastHashJoin [ca_zip,ca_zip] + Filter [ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (10) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index 08f1b447670fe..efac82f31fdb4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) :- * Project (12) : +- * BroadcastHashJoin Inner BuildRight (11) : :- * Project (6) @@ -17,32 +17,31 @@ TakeOrderedAndProject (44) : +- * Filter (9) : +- * ColumnarToRow (8) : +- Scan parquet default.store (7) - +- BroadcastExchange (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * Project (16) - : +- * Filter (15) - : +- * ColumnarToRow (14) - : +- Scan parquet default.customer_address (13) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet default.customer_address (17) - +- BroadcastExchange (24) - +- * Project (23) - +- * Filter (22) - +- * ColumnarToRow (21) - +- Scan parquet default.customer (20) + +- BroadcastExchange (37) + +- * HashAggregate (36) + +- Exchange (35) + +- * HashAggregate (34) + +- * Project (33) + +- * BroadcastHashJoin LeftSemi BuildRight (32) + :- * Filter (15) + : +- * ColumnarToRow (14) + : +- Scan parquet default.customer_address (13) + +- BroadcastExchange (31) + +- * Project (30) + +- * Filter (29) + +- * HashAggregate (28) + +- Exchange (27) + +- * HashAggregate (26) + +- * Project (25) + +- * BroadcastHashJoin Inner BuildRight (24) + :- * Filter (18) + : +- * ColumnarToRow (17) + : +- Scan parquet default.customer_address (16) + +- BroadcastExchange (23) + +- * Project (22) + +- * Filter (21) + +- * ColumnarToRow (20) + +- Scan parquet default.customer (19) (1) Scan parquet default.store_sales @@ -60,7 +59,7 @@ Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(4) ReusedExchange [Reuses operator id: 49] +(4) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 8] @@ -96,193 +95,189 @@ Right keys [1]: [s_store_sk#6] Join condition: None (12) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_store_name#7, substr(s_zip#8, 1, 2) AS substr(spark_catalog.default.store.s_zip, 1, 2)#10] +Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#8] (13) Scan parquet default.customer_address -Output [1]: [ca_zip#11] +Output [1]: [ca_zip#10] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] ReadSchema: struct (14) ColumnarToRow [codegen id : 6] -Input [1]: [ca_zip#11] +Input [1]: [ca_zip#10] (15) Filter [codegen id : 6] -Input [1]: [ca_zip#11] -Condition : (substr(ca_zip#11, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#11, 1, 5))) +Input [1]: [ca_zip#10] +Condition : (substr(ca_zip#10, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#10, 1, 5))) -(16) Project [codegen id : 6] -Output [2]: [ca_zip#11, substr(ca_zip#11, 1, 5) AS substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#12] -Input [1]: [ca_zip#11] - -(17) Scan parquet default.customer_address -Output [2]: [ca_address_sk#13, ca_zip#14] +(16) Scan parquet default.customer_address +Output [2]: [ca_address_sk#11, ca_zip#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#13, ca_zip#14] +(17) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#11, ca_zip#12] -(19) Filter [codegen id : 4] -Input [2]: [ca_address_sk#13, ca_zip#14] -Condition : isnotnull(ca_address_sk#13) +(18) Filter [codegen id : 4] +Input [2]: [ca_address_sk#11, ca_zip#12] +Condition : isnotnull(ca_address_sk#11) -(20) Scan parquet default.customer -Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +(19) Scan parquet default.customer +Output [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(21) ColumnarToRow [codegen id : 3] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +(20) ColumnarToRow [codegen id : 3] +Input [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] -(22) Filter [codegen id : 3] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Condition : ((isnotnull(c_preferred_cust_flag#16) AND (c_preferred_cust_flag#16 = Y)) AND isnotnull(c_current_addr_sk#15)) +(21) Filter [codegen id : 3] +Input [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] +Condition : ((isnotnull(c_preferred_cust_flag#14) AND (c_preferred_cust_flag#14 = Y)) AND isnotnull(c_current_addr_sk#13)) -(23) Project [codegen id : 3] -Output [1]: [c_current_addr_sk#15] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +(22) Project [codegen id : 3] +Output [1]: [c_current_addr_sk#13] +Input [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] -(24) BroadcastExchange -Input [1]: [c_current_addr_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +(23) BroadcastExchange +Input [1]: [c_current_addr_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] -(25) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ca_address_sk#13] -Right keys [1]: [c_current_addr_sk#15] +(24) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ca_address_sk#11] +Right keys [1]: [c_current_addr_sk#13] Join condition: None -(26) Project [codegen id : 4] -Output [1]: [ca_zip#14] -Input [3]: [ca_address_sk#13, ca_zip#14, c_current_addr_sk#15] +(25) Project [codegen id : 4] +Output [1]: [ca_zip#12] +Input [3]: [ca_address_sk#11, ca_zip#12, c_current_addr_sk#13] -(27) HashAggregate [codegen id : 4] -Input [1]: [ca_zip#14] -Keys [1]: [ca_zip#14] +(26) HashAggregate [codegen id : 4] +Input [1]: [ca_zip#12] +Keys [1]: [ca_zip#12] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [2]: [ca_zip#14, count#19] +Aggregate Attributes [1]: [count#16] +Results [2]: [ca_zip#12, count#17] -(28) Exchange -Input [2]: [ca_zip#14, count#19] -Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [id=#20] +(27) Exchange +Input [2]: [ca_zip#12, count#17] +Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [id=#18] -(29) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#14, count#19] -Keys [1]: [ca_zip#14] +(28) HashAggregate [codegen id : 5] +Input [2]: [ca_zip#12, count#17] +Keys [1]: [ca_zip#12] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#21] -Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#22, count(1)#21 AS cnt#23] +Aggregate Attributes [1]: [count(1)#19] +Results [2]: [substr(ca_zip#12, 1, 5) AS ca_zip#20, count(1)#19 AS cnt#21] -(30) Filter [codegen id : 5] -Input [2]: [ca_zip#22, cnt#23] -Condition : (cnt#23 > 10) +(29) Filter [codegen id : 5] +Input [2]: [ca_zip#20, cnt#21] +Condition : (cnt#21 > 10) -(31) Project [codegen id : 5] -Output [1]: [ca_zip#22] -Input [2]: [ca_zip#22, cnt#23] +(30) Project [codegen id : 5] +Output [1]: [ca_zip#20] +Input [2]: [ca_zip#20, cnt#21] -(32) BroadcastExchange -Input [1]: [ca_zip#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#24] +(31) BroadcastExchange +Input [1]: [ca_zip#20] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#22] -(33) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [coalesce(substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#12, ), isnull(substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#12)] -Right keys [2]: [coalesce(ca_zip#22, ), isnull(ca_zip#22)] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [2]: [coalesce(substr(ca_zip#10, 1, 5), ), isnull(substr(ca_zip#10, 1, 5))] +Right keys [2]: [coalesce(ca_zip#20, ), isnull(ca_zip#20)] Join condition: None -(34) Project [codegen id : 6] -Output [1]: [substr(ca_zip#11, 1, 5) AS ca_zip#25] -Input [2]: [ca_zip#11, substr(spark_catalog.default.customer_address.ca_zip, 1, 5)#12] +(33) Project [codegen id : 6] +Output [1]: [substr(ca_zip#10, 1, 5) AS ca_zip#23] +Input [1]: [ca_zip#10] -(35) HashAggregate [codegen id : 6] -Input [1]: [ca_zip#25] -Keys [1]: [ca_zip#25] +(34) HashAggregate [codegen id : 6] +Input [1]: [ca_zip#23] +Keys [1]: [ca_zip#23] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#25] +Results [1]: [ca_zip#23] -(36) Exchange -Input [1]: [ca_zip#25] -Arguments: hashpartitioning(ca_zip#25, 5), ENSURE_REQUIREMENTS, [id=#26] +(35) Exchange +Input [1]: [ca_zip#23] +Arguments: hashpartitioning(ca_zip#23, 5), ENSURE_REQUIREMENTS, [id=#24] -(37) HashAggregate [codegen id : 7] -Input [1]: [ca_zip#25] -Keys [1]: [ca_zip#25] +(36) HashAggregate [codegen id : 7] +Input [1]: [ca_zip#23] +Keys [1]: [ca_zip#23] Functions: [] Aggregate Attributes: [] -Results [1]: [substr(ca_zip#25, 1, 2) AS substr(V1.ca_zip, 1, 2)#27] +Results [1]: [ca_zip#23] -(38) BroadcastExchange -Input [1]: [substr(V1.ca_zip, 1, 2)#27] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#28] +(37) BroadcastExchange +Input [1]: [ca_zip#23] +Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [id=#25] -(39) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [substr(spark_catalog.default.store.s_zip, 1, 2)#10] -Right keys [1]: [substr(V1.ca_zip, 1, 2)#27] +(38) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [substr(s_zip#8, 1, 2)] +Right keys [1]: [substr(ca_zip#23, 1, 2)] Join condition: None -(40) Project [codegen id : 8] +(39) Project [codegen id : 8] Output [2]: [ss_net_profit#2, s_store_name#7] -Input [4]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10, substr(V1.ca_zip, 1, 2)#27] +Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#8, ca_zip#23] -(41) HashAggregate [codegen id : 8] +(40) HashAggregate [codegen id : 8] Input [2]: [ss_net_profit#2, s_store_name#7] Keys [1]: [s_store_name#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#29] -Results [2]: [s_store_name#7, sum#30] +Aggregate Attributes [1]: [sum#26] +Results [2]: [s_store_name#7, sum#27] -(42) Exchange -Input [2]: [s_store_name#7, sum#30] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#31] +(41) Exchange +Input [2]: [s_store_name#7, sum#27] +Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#28] -(43) HashAggregate [codegen id : 9] -Input [2]: [s_store_name#7, sum#30] +(42) HashAggregate [codegen id : 9] +Input [2]: [s_store_name#7, sum#27] Keys [1]: [s_store_name#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#32] -Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#32,17,2) AS sum(ss_net_profit)#33] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#29] +Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#29,17,2) AS sum(ss_net_profit)#30] -(44) TakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#33] -Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#33] +(43) TakeOrderedAndProject +Input [2]: [s_store_name#7, sum(ss_net_profit)#30] +Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * Project (48) - +- * Filter (47) - +- * ColumnarToRow (46) - +- Scan parquet default.date_dim (45) +BroadcastExchange (48) ++- * Project (47) + +- * Filter (46) + +- * ColumnarToRow (45) + +- Scan parquet default.date_dim (44) -(45) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#34, d_qoy#35] +(44) Scan parquet default.date_dim +Output [3]: [d_date_sk#5, d_year#31, d_qoy#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#34, d_qoy#35] +(45) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#5, d_year#31, d_qoy#32] -(47) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#34, d_qoy#35] -Condition : ((((isnotnull(d_qoy#35) AND isnotnull(d_year#34)) AND (d_qoy#35 = 2)) AND (d_year#34 = 1998)) AND isnotnull(d_date_sk#5)) +(46) Filter [codegen id : 1] +Input [3]: [d_date_sk#5, d_year#31, d_qoy#32] +Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 2)) AND (d_year#31 = 1998)) AND isnotnull(d_date_sk#5)) -(48) Project [codegen id : 1] +(47) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#34, d_qoy#35] +Input [3]: [d_date_sk#5, d_year#31, d_qoy#32] -(49) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index 94ac2b90c3201..a28dcb87ee2e4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (8) HashAggregate [s_store_name,ss_net_profit] [sum,sum] Project [ss_net_profit,s_store_name] - BroadcastHashJoin [substr(spark_catalog.default.store.s_zip, 1, 2),substr(V1.ca_zip, 1, 2)] + BroadcastHashJoin [s_zip,ca_zip] Project [ss_net_profit,s_store_name,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk,ss_net_profit] @@ -35,18 +35,17 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) - HashAggregate [ca_zip] [substr(V1.ca_zip, 1, 2)] + HashAggregate [ca_zip] InputAdapter Exchange [ca_zip] #5 WholeStageCodegen (6) HashAggregate [ca_zip] Project [ca_zip] - BroadcastHashJoin [substr(spark_catalog.default.customer_address.ca_zip, 1, 5),ca_zip] - Project [ca_zip] - Filter [ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_zip] + BroadcastHashJoin [ca_zip,ca_zip] + Filter [ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt index 130a57b39e1b7..84c2fd54d5f4b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt @@ -209,112 +209,112 @@ Right keys [1]: [ctr_customer_sk#35] Join condition: None (34) Project [codegen id : 20] -Output [18]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#36, ctr_total_return#37, cast(ctr_total_return#37 as decimal(24,7)) AS CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#39] +Output [17]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#36, ctr_total_return#37] Input [19]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_customer_sk#35, ctr_state#36, ctr_total_return#37] (35) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42, cr_returned_date_sk#43] +Output [4]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, cr_returned_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#43), dynamicpruningexpression(cr_returned_date_sk#43 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cr_returned_date_sk#42), dynamicpruningexpression(cr_returned_date_sk#42 IN dynamicpruning#25)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 13] -Input [4]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42, cr_returned_date_sk#43] +Input [4]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, cr_returned_date_sk#42] (37) Filter [codegen id : 13] -Input [4]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42, cr_returned_date_sk#43] -Condition : isnotnull(cr_returning_addr_sk#41) +Input [4]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, cr_returned_date_sk#42] +Condition : isnotnull(cr_returning_addr_sk#40) (38) ReusedExchange [Reuses operator id: 62] -Output [1]: [d_date_sk#44] +Output [1]: [d_date_sk#43] (39) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#43] -Right keys [1]: [d_date_sk#44] +Left keys [1]: [cr_returned_date_sk#42] +Right keys [1]: [d_date_sk#43] Join condition: None (40) Project [codegen id : 13] -Output [3]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42] -Input [5]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42, cr_returned_date_sk#43, d_date_sk#44] +Output [3]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41] +Input [5]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, cr_returned_date_sk#42, d_date_sk#43] (41) Exchange -Input [3]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42] -Arguments: hashpartitioning(cr_returning_addr_sk#41, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [3]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41] +Arguments: hashpartitioning(cr_returning_addr_sk#40, 5), ENSURE_REQUIREMENTS, [id=#44] (42) Sort [codegen id : 14] -Input [3]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42] -Arguments: [cr_returning_addr_sk#41 ASC NULLS FIRST], false, 0 +Input [3]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41] +Arguments: [cr_returning_addr_sk#40 ASC NULLS FIRST], false, 0 (43) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#46, ca_state#47] +Output [2]: [ca_address_sk#45, ca_state#46] (44) Sort [codegen id : 16] -Input [2]: [ca_address_sk#46, ca_state#47] -Arguments: [ca_address_sk#46 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#45, ca_state#46] +Arguments: [ca_address_sk#45 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] -Left keys [1]: [cr_returning_addr_sk#41] -Right keys [1]: [ca_address_sk#46] +Left keys [1]: [cr_returning_addr_sk#40] +Right keys [1]: [ca_address_sk#45] Join condition: None (46) Project [codegen id : 17] -Output [3]: [cr_returning_customer_sk#40, cr_return_amt_inc_tax#42, ca_state#47] -Input [5]: [cr_returning_customer_sk#40, cr_returning_addr_sk#41, cr_return_amt_inc_tax#42, ca_address_sk#46, ca_state#47] +Output [3]: [cr_returning_customer_sk#39, cr_return_amt_inc_tax#41, ca_state#46] +Input [5]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, ca_address_sk#45, ca_state#46] (47) HashAggregate [codegen id : 17] -Input [3]: [cr_returning_customer_sk#40, cr_return_amt_inc_tax#42, ca_state#47] -Keys [2]: [cr_returning_customer_sk#40, ca_state#47] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#42))] -Aggregate Attributes [1]: [sum#48] -Results [3]: [cr_returning_customer_sk#40, ca_state#47, sum#49] +Input [3]: [cr_returning_customer_sk#39, cr_return_amt_inc_tax#41, ca_state#46] +Keys [2]: [cr_returning_customer_sk#39, ca_state#46] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#41))] +Aggregate Attributes [1]: [sum#47] +Results [3]: [cr_returning_customer_sk#39, ca_state#46, sum#48] (48) Exchange -Input [3]: [cr_returning_customer_sk#40, ca_state#47, sum#49] -Arguments: hashpartitioning(cr_returning_customer_sk#40, ca_state#47, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [cr_returning_customer_sk#39, ca_state#46, sum#48] +Arguments: hashpartitioning(cr_returning_customer_sk#39, ca_state#46, 5), ENSURE_REQUIREMENTS, [id=#49] (49) HashAggregate [codegen id : 18] -Input [3]: [cr_returning_customer_sk#40, ca_state#47, sum#49] -Keys [2]: [cr_returning_customer_sk#40, ca_state#47] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#42))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#42))#51] -Results [2]: [ca_state#47 AS ctr_state#36, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#42))#51,17,2) AS ctr_total_return#37] +Input [3]: [cr_returning_customer_sk#39, ca_state#46, sum#48] +Keys [2]: [cr_returning_customer_sk#39, ca_state#46] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#41))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#41))#50] +Results [2]: [ca_state#46 AS ctr_state#36, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#41))#50,17,2) AS ctr_total_return#37] (50) HashAggregate [codegen id : 18] Input [2]: [ctr_state#36, ctr_total_return#37] Keys [1]: [ctr_state#36] Functions [1]: [partial_avg(ctr_total_return#37)] -Aggregate Attributes [2]: [sum#52, count#53] -Results [3]: [ctr_state#36, sum#54, count#55] +Aggregate Attributes [2]: [sum#51, count#52] +Results [3]: [ctr_state#36, sum#53, count#54] (51) Exchange -Input [3]: [ctr_state#36, sum#54, count#55] -Arguments: hashpartitioning(ctr_state#36, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [3]: [ctr_state#36, sum#53, count#54] +Arguments: hashpartitioning(ctr_state#36, 5), ENSURE_REQUIREMENTS, [id=#55] (52) HashAggregate [codegen id : 19] -Input [3]: [ctr_state#36, sum#54, count#55] +Input [3]: [ctr_state#36, sum#53, count#54] Keys [1]: [ctr_state#36] Functions [1]: [avg(ctr_total_return#37)] -Aggregate Attributes [1]: [avg(ctr_total_return#37)#57] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#37)#57) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#58, ctr_state#36 AS ctr_state#36#59] +Aggregate Attributes [1]: [avg(ctr_total_return#37)#56] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#37)#56) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#57, ctr_state#36 AS ctr_state#36#58] (53) Filter [codegen id : 19] -Input [2]: [(avg(ctr_total_return) * 1.2)#58, ctr_state#36#59] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#58) +Input [2]: [(avg(ctr_total_return) * 1.2)#57, ctr_state#36#58] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#57) (54) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#58, ctr_state#36#59] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#60] +Input [2]: [(avg(ctr_total_return) * 1.2)#57, ctr_state#36#58] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#59] (55) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ctr_state#36] -Right keys [1]: [ctr_state#36#59] -Join condition: (CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#39 > (avg(ctr_total_return) * 1.2)#58) +Right keys [1]: [ctr_state#36#58] +Join condition: (cast(ctr_total_return#37 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#57) (56) Project [codegen id : 20] Output [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#37] -Input [20]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#36, ctr_total_return#37, CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#39, (avg(ctr_total_return) * 1.2)#58, ctr_state#36#59] +Input [19]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#36, ctr_total_return#37, (avg(ctr_total_return) * 1.2)#57, ctr_state#36#58] (57) TakeOrderedAndProject Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#37] @@ -331,27 +331,27 @@ BroadcastExchange (62) (58) Scan parquet default.date_dim -Output [2]: [d_date_sk#26, d_year#61] +Output [2]: [d_date_sk#26, d_year#60] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#26, d_year#61] +Input [2]: [d_date_sk#26, d_year#60] (60) Filter [codegen id : 1] -Input [2]: [d_date_sk#26, d_year#61] -Condition : ((isnotnull(d_year#61) AND (d_year#61 = 2000)) AND isnotnull(d_date_sk#26)) +Input [2]: [d_date_sk#26, d_year#60] +Condition : ((isnotnull(d_year#60) AND (d_year#60 = 2000)) AND isnotnull(d_date_sk#26)) (61) Project [codegen id : 1] Output [1]: [d_date_sk#26] -Input [2]: [d_date_sk#26, d_year#61] +Input [2]: [d_date_sk#26, d_year#60] (62) BroadcastExchange Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] -Subquery:2 Hosting operator id = 35 Hosting Expression = cr_returned_date_sk#43 IN dynamicpruning#25 +Subquery:2 Hosting operator id = 35 Hosting Expression = cr_returned_date_sk#42 IN dynamicpruning#25 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt index bbe758caf12b2..bca54597ee97f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] WholeStageCodegen (20) Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,CAST(ctr1.ctr_total_return AS DECIMAL(24,7)),(avg(ctr_total_return) * 1.2)] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_state,ctr_total_return] SortMergeJoin [c_customer_sk,ctr_customer_sk] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt index 264674f7e825b..4a9a9cc153d0e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt @@ -1,53 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- * BroadcastHashJoin Inner BuildRight (47) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Project (17) - : : : +- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.catalog_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.customer_address (7) - : : +- BroadcastExchange (34) - : : +- * Filter (33) - : : +- * HashAggregate (32) - : : +- Exchange (31) - : : +- * HashAggregate (30) - : : +- * HashAggregate (29) - : : +- Exchange (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * Filter (20) - : : : : +- * ColumnarToRow (19) - : : : : +- Scan parquet default.catalog_returns (18) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- BroadcastExchange (40) - : +- * Filter (39) - : +- * ColumnarToRow (38) - : +- Scan parquet default.customer (37) - +- BroadcastExchange (46) - +- * Filter (45) - +- * ColumnarToRow (44) - +- Scan parquet default.customer_address (43) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.catalog_returns (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.customer_address (7) + : : +- BroadcastExchange (33) + : : +- * Filter (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * HashAggregate (28) + : : +- Exchange (27) + : : +- * HashAggregate (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * Filter (19) + : : : : +- * ColumnarToRow (18) + : : : : +- Scan parquet default.catalog_returns (17) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : +- BroadcastExchange (39) + : +- * Filter (38) + : +- * ColumnarToRow (37) + : +- Scan parquet default.customer (36) + +- BroadcastExchange (45) + +- * Filter (44) + +- * ColumnarToRow (43) + +- Scan parquet default.customer_address (42) (1) Scan parquet default.catalog_returns @@ -65,7 +64,7 @@ Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_in Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) -(4) ReusedExchange [Reuses operator id: 54] +(4) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -126,192 +125,188 @@ Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#14, ca_state#8 AS ct Input [3]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16] Condition : isnotnull(ctr_total_return#16) -(17) Project [codegen id : 11] -Output [4]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, cast(ctr_total_return#16 as decimal(24,7)) AS CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#17] -Input [3]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16] - -(18) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#18, cr_returning_addr_sk#19, cr_return_amt_inc_tax#20, cr_returned_date_sk#21] +(17) Scan parquet default.catalog_returns +Output [4]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, cr_returned_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#21), dynamicpruningexpression(cr_returned_date_sk#21 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#20), dynamicpruningexpression(cr_returned_date_sk#20 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returning_customer_sk#18, cr_returning_addr_sk#19, cr_return_amt_inc_tax#20, cr_returned_date_sk#21] +(18) ColumnarToRow [codegen id : 6] +Input [4]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, cr_returned_date_sk#20] -(20) Filter [codegen id : 6] -Input [4]: [cr_returning_customer_sk#18, cr_returning_addr_sk#19, cr_return_amt_inc_tax#20, cr_returned_date_sk#21] -Condition : isnotnull(cr_returning_addr_sk#19) +(19) Filter [codegen id : 6] +Input [4]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, cr_returned_date_sk#20] +Condition : isnotnull(cr_returning_addr_sk#18) -(21) ReusedExchange [Reuses operator id: 54] -Output [1]: [d_date_sk#22] +(20) ReusedExchange [Reuses operator id: 53] +Output [1]: [d_date_sk#21] -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#21] -Right keys [1]: [d_date_sk#22] +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cr_returned_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None -(23) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#18, cr_returning_addr_sk#19, cr_return_amt_inc_tax#20] -Input [5]: [cr_returning_customer_sk#18, cr_returning_addr_sk#19, cr_return_amt_inc_tax#20, cr_returned_date_sk#21, d_date_sk#22] +(22) Project [codegen id : 6] +Output [3]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19] +Input [5]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, cr_returned_date_sk#20, d_date_sk#21] -(24) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#23, ca_state#24] +(23) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#22, ca_state#23] -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#19] -Right keys [1]: [ca_address_sk#23] +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cr_returning_addr_sk#18] +Right keys [1]: [ca_address_sk#22] Join condition: None -(26) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#18, cr_return_amt_inc_tax#20, ca_state#24] -Input [5]: [cr_returning_customer_sk#18, cr_returning_addr_sk#19, cr_return_amt_inc_tax#20, ca_address_sk#23, ca_state#24] +(25) Project [codegen id : 6] +Output [3]: [cr_returning_customer_sk#17, cr_return_amt_inc_tax#19, ca_state#23] +Input [5]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, ca_address_sk#22, ca_state#23] -(27) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#18, cr_return_amt_inc_tax#20, ca_state#24] -Keys [2]: [cr_returning_customer_sk#18, ca_state#24] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#20))] -Aggregate Attributes [1]: [sum#25] -Results [3]: [cr_returning_customer_sk#18, ca_state#24, sum#26] +(26) HashAggregate [codegen id : 6] +Input [3]: [cr_returning_customer_sk#17, cr_return_amt_inc_tax#19, ca_state#23] +Keys [2]: [cr_returning_customer_sk#17, ca_state#23] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#19))] +Aggregate Attributes [1]: [sum#24] +Results [3]: [cr_returning_customer_sk#17, ca_state#23, sum#25] -(28) Exchange -Input [3]: [cr_returning_customer_sk#18, ca_state#24, sum#26] -Arguments: hashpartitioning(cr_returning_customer_sk#18, ca_state#24, 5), ENSURE_REQUIREMENTS, [id=#27] +(27) Exchange +Input [3]: [cr_returning_customer_sk#17, ca_state#23, sum#25] +Arguments: hashpartitioning(cr_returning_customer_sk#17, ca_state#23, 5), ENSURE_REQUIREMENTS, [id=#26] -(29) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#18, ca_state#24, sum#26] -Keys [2]: [cr_returning_customer_sk#18, ca_state#24] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#20))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#20))#28] -Results [2]: [ca_state#24 AS ctr_state#15, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#20))#28,17,2) AS ctr_total_return#16] +(28) HashAggregate [codegen id : 7] +Input [3]: [cr_returning_customer_sk#17, ca_state#23, sum#25] +Keys [2]: [cr_returning_customer_sk#17, ca_state#23] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#19))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#19))#27] +Results [2]: [ca_state#23 AS ctr_state#15, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#19))#27,17,2) AS ctr_total_return#16] -(30) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 7] Input [2]: [ctr_state#15, ctr_total_return#16] Keys [1]: [ctr_state#15] Functions [1]: [partial_avg(ctr_total_return#16)] -Aggregate Attributes [2]: [sum#29, count#30] -Results [3]: [ctr_state#15, sum#31, count#32] +Aggregate Attributes [2]: [sum#28, count#29] +Results [3]: [ctr_state#15, sum#30, count#31] -(31) Exchange -Input [3]: [ctr_state#15, sum#31, count#32] -Arguments: hashpartitioning(ctr_state#15, 5), ENSURE_REQUIREMENTS, [id=#33] +(30) Exchange +Input [3]: [ctr_state#15, sum#30, count#31] +Arguments: hashpartitioning(ctr_state#15, 5), ENSURE_REQUIREMENTS, [id=#32] -(32) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#15, sum#31, count#32] +(31) HashAggregate [codegen id : 8] +Input [3]: [ctr_state#15, sum#30, count#31] Keys [1]: [ctr_state#15] Functions [1]: [avg(ctr_total_return#16)] -Aggregate Attributes [1]: [avg(ctr_total_return#16)#34] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#16)#34) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#35, ctr_state#15 AS ctr_state#15#36] +Aggregate Attributes [1]: [avg(ctr_total_return#16)#33] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#16)#33) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#34, ctr_state#15 AS ctr_state#15#35] -(33) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#35, ctr_state#15#36] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#35) +(32) Filter [codegen id : 8] +Input [2]: [(avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#34) -(34) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#35, ctr_state#15#36] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#37] +(33) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#36] -(35) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#15] -Right keys [1]: [ctr_state#15#36] -Join condition: (CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#17 > (avg(ctr_total_return) * 1.2)#35) +Right keys [1]: [ctr_state#15#35] +Join condition: (cast(ctr_total_return#16 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#34) -(36) Project [codegen id : 11] +(35) Project [codegen id : 11] Output [2]: [ctr_customer_sk#14, ctr_total_return#16] -Input [6]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, CAST(ctr1.ctr_total_return AS DECIMAL(24,7))#17, (avg(ctr_total_return) * 1.2)#35, ctr_state#15#36] +Input [5]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, (avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] -(37) Scan parquet default.customer -Output [6]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43] +(36) Scan parquet default.customer +Output [6]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43] +(37) ColumnarToRow [codegen id : 9] +Input [6]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] -(39) Filter [codegen id : 9] -Input [6]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43] -Condition : (isnotnull(c_customer_sk#38) AND isnotnull(c_current_addr_sk#40)) +(38) Filter [codegen id : 9] +Input [6]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] +Condition : (isnotnull(c_customer_sk#37) AND isnotnull(c_current_addr_sk#39)) -(40) BroadcastExchange -Input [6]: [c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] +(39) BroadcastExchange +Input [6]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] -(41) BroadcastHashJoin [codegen id : 11] +(40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#14] -Right keys [1]: [c_customer_sk#38] +Right keys [1]: [c_customer_sk#37] Join condition: None -(42) Project [codegen id : 11] -Output [6]: [ctr_total_return#16, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43] -Input [8]: [ctr_customer_sk#14, ctr_total_return#16, c_customer_sk#38, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43] +(41) Project [codegen id : 11] +Output [6]: [ctr_total_return#16, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] +Input [8]: [ctr_customer_sk#14, ctr_total_return#16, c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] -(43) Scan parquet default.customer_address -Output [12]: [ca_address_sk#45, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56] +(42) Scan parquet default.customer_address +Output [12]: [ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#45, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56] +(43) ColumnarToRow [codegen id : 10] +Input [12]: [ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] -(45) Filter [codegen id : 10] -Input [12]: [ca_address_sk#45, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56] -Condition : ((isnotnull(ca_state#52) AND (ca_state#52 = GA)) AND isnotnull(ca_address_sk#45)) +(44) Filter [codegen id : 10] +Input [12]: [ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] +Condition : ((isnotnull(ca_state#51) AND (ca_state#51 = GA)) AND isnotnull(ca_address_sk#44)) -(46) BroadcastExchange -Input [12]: [ca_address_sk#45, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#57] +(45) BroadcastExchange +Input [12]: [ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#56] -(47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#45] +(46) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#39] +Right keys [1]: [ca_address_sk#44] Join condition: None -(48) Project [codegen id : 11] -Output [16]: [c_customer_id#39, c_salutation#41, c_first_name#42, c_last_name#43, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56, ctr_total_return#16] -Input [18]: [ctr_total_return#16, c_customer_id#39, c_current_addr_sk#40, c_salutation#41, c_first_name#42, c_last_name#43, ca_address_sk#45, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56] +(47) Project [codegen id : 11] +Output [16]: [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55, ctr_total_return#16] +Input [18]: [ctr_total_return#16, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] -(49) TakeOrderedAndProject -Input [16]: [c_customer_id#39, c_salutation#41, c_first_name#42, c_last_name#43, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56, ctr_total_return#16] -Arguments: 100, [c_customer_id#39 ASC NULLS FIRST, c_salutation#41 ASC NULLS FIRST, c_first_name#42 ASC NULLS FIRST, c_last_name#43 ASC NULLS FIRST, ca_street_number#46 ASC NULLS FIRST, ca_street_name#47 ASC NULLS FIRST, ca_street_type#48 ASC NULLS FIRST, ca_suite_number#49 ASC NULLS FIRST, ca_city#50 ASC NULLS FIRST, ca_county#51 ASC NULLS FIRST, ca_state#52 ASC NULLS FIRST, ca_zip#53 ASC NULLS FIRST, ca_country#54 ASC NULLS FIRST, ca_gmt_offset#55 ASC NULLS FIRST, ca_location_type#56 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#39, c_salutation#41, c_first_name#42, c_last_name#43, ca_street_number#46, ca_street_name#47, ca_street_type#48, ca_suite_number#49, ca_city#50, ca_county#51, ca_state#52, ca_zip#53, ca_country#54, ca_gmt_offset#55, ca_location_type#56, ctr_total_return#16] +(48) TakeOrderedAndProject +Input [16]: [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55, ctr_total_return#16] +Arguments: 100, [c_customer_id#38 ASC NULLS FIRST, c_salutation#40 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, c_last_name#42 ASC NULLS FIRST, ca_street_number#45 ASC NULLS FIRST, ca_street_name#46 ASC NULLS FIRST, ca_street_type#47 ASC NULLS FIRST, ca_suite_number#48 ASC NULLS FIRST, ca_city#49 ASC NULLS FIRST, ca_county#50 ASC NULLS FIRST, ca_state#51 ASC NULLS FIRST, ca_zip#52 ASC NULLS FIRST, ca_country#53 ASC NULLS FIRST, ca_gmt_offset#54 ASC NULLS FIRST, ca_location_type#55 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55, ctr_total_return#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (54) -+- * Project (53) - +- * Filter (52) - +- * ColumnarToRow (51) - +- Scan parquet default.date_dim (50) +BroadcastExchange (53) ++- * Project (52) + +- * Filter (51) + +- * ColumnarToRow (50) + +- Scan parquet default.date_dim (49) -(50) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#58] +(49) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_year#57] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(51) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#58] +(50) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#57] -(52) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#58] -Condition : ((isnotnull(d_year#58) AND (d_year#58 = 2000)) AND isnotnull(d_date_sk#6)) +(51) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_year#57] +Condition : ((isnotnull(d_year#57) AND (d_year#57 = 2000)) AND isnotnull(d_date_sk#6)) -(53) Project [codegen id : 1] +(52) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#58] +Input [2]: [d_date_sk#6, d_year#57] -(54) BroadcastExchange +(53) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] -Subquery:2 Hosting operator id = 18 Hosting Expression = cr_returned_date_sk#21 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = cr_returned_date_sk#20 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt index 354d405ee12c0..af8c23b5f7a00 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -5,39 +5,38 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,CAST(ctr1.ctr_total_return AS DECIMAL(24,7)),(avg(ctr_total_return) * 1.2)] - Project [ctr_customer_sk,ctr_state,ctr_total_return] - Filter [ctr_total_return] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returning_addr_sk,cr_returning_customer_sk] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] + Filter [ctr_total_return] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] + InputAdapter + Exchange [cr_returning_customer_sk,ca_state] #1 + WholeStageCodegen (3) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Filter [cr_returning_addr_sk,cr_returning_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Filter [ca_address_sk,ca_state] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [ca_address_sk,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt index 3b653ecf5d846..d13b0f1c9bb91 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == -* HashAggregate (30) -+- Exchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildLeft (23) +* HashAggregate (29) ++- Exchange (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * Project (23) + : +- * BroadcastHashJoin Inner BuildLeft (22) : :- BroadcastExchange (18) : : +- * Project (17) : : +- * BroadcastHashJoin Inner BuildLeft (16) @@ -24,11 +24,10 @@ : : : +- * ColumnarToRow (7) : : : +- Scan parquet default.web_sales (6) : : +- ReusedExchange (9) - : +- * Project (22) - : +- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet default.web_sales (19) - +- ReusedExchange (25) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.web_sales (19) + +- ReusedExchange (24) (1) Scan parquet default.item @@ -68,7 +67,7 @@ Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] Condition : isnotnull(ws_item_sk#4) -(9) ReusedExchange [Reuses operator id: 35] +(9) ReusedExchange [Reuses operator id: 34] Output [1]: [d_date_sk#8] (10) BroadcastHashJoin [codegen id : 3] @@ -130,80 +129,76 @@ Input [3]: [ws_item_sk#17, ws_ext_discount_amt#18, ws_sold_date_sk#19] Input [3]: [ws_item_sk#17, ws_ext_discount_amt#18, ws_sold_date_sk#19] Condition : (isnotnull(ws_item_sk#17) AND isnotnull(ws_ext_discount_amt#18)) -(22) Project -Output [4]: [ws_item_sk#17, ws_ext_discount_amt#18, ws_sold_date_sk#19, cast(ws_ext_discount_amt#18 as decimal(14,7)) AS CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7))#20] -Input [3]: [ws_item_sk#17, ws_ext_discount_amt#18, ws_sold_date_sk#19] - -(23) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] Right keys [1]: [ws_item_sk#17] -Join condition: (CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7))#20 > (1.3 * avg(ws_ext_discount_amt))#15) +Join condition: (cast(ws_ext_discount_amt#18 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#15) -(24) Project [codegen id : 6] +(23) Project [codegen id : 6] Output [2]: [ws_ext_discount_amt#18, ws_sold_date_sk#19] -Input [6]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#17, ws_ext_discount_amt#18, ws_sold_date_sk#19, CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7))#20] +Input [5]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#17, ws_ext_discount_amt#18, ws_sold_date_sk#19] -(25) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#21] +(24) ReusedExchange [Reuses operator id: 34] +Output [1]: [d_date_sk#20] -(26) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#19] -Right keys [1]: [d_date_sk#21] +Right keys [1]: [d_date_sk#20] Join condition: None -(27) Project [codegen id : 6] +(26) Project [codegen id : 6] Output [1]: [ws_ext_discount_amt#18] -Input [3]: [ws_ext_discount_amt#18, ws_sold_date_sk#19, d_date_sk#21] +Input [3]: [ws_ext_discount_amt#18, ws_sold_date_sk#19, d_date_sk#20] -(28) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 6] Input [1]: [ws_ext_discount_amt#18] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#18))] -Aggregate Attributes [1]: [sum#22] -Results [1]: [sum#23] +Aggregate Attributes [1]: [sum#21] +Results [1]: [sum#22] -(29) Exchange -Input [1]: [sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +(28) Exchange +Input [1]: [sum#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] -(30) HashAggregate [codegen id : 7] -Input [1]: [sum#23] +(29) HashAggregate [codegen id : 7] +Input [1]: [sum#22] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#18))#25] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#18))#25,17,2) AS Excess Discount Amount #26] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#18))#24] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#18))#24,17,2) AS Excess Discount Amount #25] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (35) -+- * Project (34) - +- * Filter (33) - +- * ColumnarToRow (32) - +- Scan parquet default.date_dim (31) +BroadcastExchange (34) ++- * Project (33) + +- * Filter (32) + +- * ColumnarToRow (31) + +- Scan parquet default.date_dim (30) -(31) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#27] +(30) Scan parquet default.date_dim +Output [2]: [d_date_sk#8, d_date#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#8, d_date#27] +(31) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#8, d_date#26] -(33) Filter [codegen id : 1] -Input [2]: [d_date_sk#8, d_date#27] -Condition : (((isnotnull(d_date#27) AND (d_date#27 >= 2000-01-27)) AND (d_date#27 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) +(32) Filter [codegen id : 1] +Input [2]: [d_date_sk#8, d_date#26] +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 2000-01-27)) AND (d_date#26 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) -(34) Project [codegen id : 1] +(33) Project [codegen id : 1] Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#27] +Input [2]: [d_date_sk#8, d_date#26] -(35) BroadcastExchange +(34) BroadcastExchange Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt index 03b8cf105e863..abf6a164982a1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt @@ -7,7 +7,7 @@ WholeStageCodegen (7) Project [ws_ext_discount_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_ext_discount_amt,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk,CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7)),(1.3 * avg(ws_ext_discount_amt))] + BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] InputAdapter BroadcastExchange #2 WholeStageCodegen (4) @@ -43,11 +43,10 @@ WholeStageCodegen (7) Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #5 - Project [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - Filter [ws_item_sk,ws_ext_discount_amt] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_item_sk,ws_ext_discount_amt] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index 9246fd0f322cb..72c206a372644 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -73,100 +73,100 @@ Right keys [1]: [i_item_sk#5] Join condition: None (10) Project [codegen id : 6] -Output [4]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, cast(ws_ext_discount_amt#2 as decimal(14,7)) AS CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7))#8] +Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.web_sales -Output [3]: [ws_item_sk#9, ws_ext_discount_amt#10, ws_sold_date_sk#11] +Output [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#9, ws_ext_discount_amt#10, ws_sold_date_sk#11] +Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] (13) Filter [codegen id : 3] -Input [3]: [ws_item_sk#9, ws_ext_discount_amt#10, ws_sold_date_sk#11] -Condition : isnotnull(ws_item_sk#9) +Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] +Condition : isnotnull(ws_item_sk#8) (14) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#11] (15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#10] +Right keys [1]: [d_date_sk#11] Join condition: None (16) Project [codegen id : 3] -Output [2]: [ws_item_sk#9, ws_ext_discount_amt#10] -Input [4]: [ws_item_sk#9, ws_ext_discount_amt#10, ws_sold_date_sk#11, d_date_sk#12] +Output [2]: [ws_item_sk#8, ws_ext_discount_amt#9] +Input [4]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10, d_date_sk#11] (17) HashAggregate [codegen id : 3] -Input [2]: [ws_item_sk#9, ws_ext_discount_amt#10] -Keys [1]: [ws_item_sk#9] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#10))] -Aggregate Attributes [2]: [sum#13, count#14] -Results [3]: [ws_item_sk#9, sum#15, count#16] +Input [2]: [ws_item_sk#8, ws_ext_discount_amt#9] +Keys [1]: [ws_item_sk#8] +Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#9))] +Aggregate Attributes [2]: [sum#12, count#13] +Results [3]: [ws_item_sk#8, sum#14, count#15] (18) Exchange -Input [3]: [ws_item_sk#9, sum#15, count#16] -Arguments: hashpartitioning(ws_item_sk#9, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [3]: [ws_item_sk#8, sum#14, count#15] +Arguments: hashpartitioning(ws_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#16] (19) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#9, sum#15, count#16] -Keys [1]: [ws_item_sk#9] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#10))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#10))#18] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#10))#18 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#19, ws_item_sk#9] +Input [3]: [ws_item_sk#8, sum#14, count#15] +Keys [1]: [ws_item_sk#8] +Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))] +Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))#17] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#9))#17 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#18, ws_item_sk#8] (20) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#19, ws_item_sk#9] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#19) +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#18, ws_item_sk#8] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#18) (21) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#19, ws_item_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#20] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#18, ws_item_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] (22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [ws_item_sk#9] -Join condition: (CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7))#8 > (1.3 * avg(ws_ext_discount_amt))#19) +Right keys [1]: [ws_item_sk#8] +Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#18) (23) Project [codegen id : 6] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [6]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7))#8, (1.3 * avg(ws_ext_discount_amt))#19, ws_item_sk#9] +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#18, ws_item_sk#8] (24) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#21] +Output [1]: [d_date_sk#20] (25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#21] +Right keys [1]: [d_date_sk#20] Join condition: None (26) Project [codegen id : 6] Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#21] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#20] (27) HashAggregate [codegen id : 6] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#22] -Results [1]: [sum#23] +Aggregate Attributes [1]: [sum#21] +Results [1]: [sum#22] (28) Exchange -Input [1]: [sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [1]: [sum#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] (29) HashAggregate [codegen id : 7] -Input [1]: [sum#23] +Input [1]: [sum#22] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#25] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#25,17,2) AS Excess Discount Amount #26] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#24] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#24,17,2) AS Excess Discount Amount #25] ===== Subqueries ===== @@ -179,27 +179,27 @@ BroadcastExchange (34) (30) Scan parquet default.date_dim -Output [2]: [d_date_sk#21, d_date#27] +Output [2]: [d_date_sk#20, d_date#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#21, d_date#27] +Input [2]: [d_date_sk#20, d_date#26] (32) Filter [codegen id : 1] -Input [2]: [d_date_sk#21, d_date#27] -Condition : (((isnotnull(d_date#27) AND (d_date#27 >= 2000-01-27)) AND (d_date#27 <= 2000-04-26)) AND isnotnull(d_date_sk#21)) +Input [2]: [d_date_sk#20, d_date#26] +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 2000-01-27)) AND (d_date#26 <= 2000-04-26)) AND isnotnull(d_date_sk#20)) (33) Project [codegen id : 1] -Output [1]: [d_date_sk#21] -Input [2]: [d_date_sk#21, d_date#27] +Output [1]: [d_date_sk#20] +Input [2]: [d_date_sk#20, d_date#26] (34) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [d_date_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index 291f18fe08f60..86ade79511d56 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -7,7 +7,7 @@ WholeStageCodegen (7) Project [ws_ext_discount_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_ext_discount_amt,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk,CAST(spark_catalog.default.web_sales.ws_ext_discount_amt AS DECIMAL(14,7)),(1.3 * avg(ws_ext_discount_amt))] + BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] Project [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_ext_discount_amt] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt index bd1baae66cff9..da1fd1cd155aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt @@ -320,93 +320,93 @@ Right keys [1]: [customer_id#70] Join condition: None (53) Project [codegen id : 16] -Output [7]: [customer_id#21, customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#71, CASE WHEN (year_total#22 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#22)), DecimalType(38,20), true) ELSE 0E-20 END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#73] +Output [8]: [customer_id#21, year_total#22, customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#47, year_total#71] Input [9]: [customer_id#21, year_total#22, customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#47, customer_id#70, year_total#71] (54) Scan parquet default.customer -Output [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] +Output [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (55) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] +Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] (56) Filter [codegen id : 14] -Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] -Condition : (isnotnull(c_customer_sk#74) AND isnotnull(c_customer_id#75)) +Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_customer_id#74)) (57) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] +Output [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_sold_date_sk#85 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(ws_sold_date_sk#84), dynamicpruningexpression(ws_sold_date_sk#84 IN dynamicpruning#35)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] +Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] (59) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] -Condition : isnotnull(ws_bill_customer_sk#82) +Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Condition : isnotnull(ws_bill_customer_sk#81) (60) BroadcastExchange -Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#86] +Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#85] (61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#74] -Right keys [1]: [ws_bill_customer_sk#82] +Left keys [1]: [c_customer_sk#73] +Right keys [1]: [ws_bill_customer_sk#81] Join condition: None (62) Project [codegen id : 14] -Output [10]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] -Input [12]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] +Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Input [12]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] (63) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#87, d_year#88] +Output [2]: [d_date_sk#86, d_year#87] (64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#85] -Right keys [1]: [d_date_sk#87] +Left keys [1]: [ws_sold_date_sk#84] +Right keys [1]: [d_date_sk#86] Join condition: None (65) Project [codegen id : 14] -Output [10]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, ws_ext_discount_amt#83, ws_ext_list_price#84, d_year#88] -Input [12]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85, d_date_sk#87, d_year#88] +Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] +Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84, d_date_sk#86, d_year#87] (66) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, ws_ext_discount_amt#83, ws_ext_list_price#84, d_year#88] -Keys [8]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, d_year#88] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#84 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#83 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#89] -Results [9]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, d_year#88, sum#90] +Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#88] +Results [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] (67) Exchange -Input [9]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, d_year#88, sum#90] -Arguments: hashpartitioning(c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, d_year#88, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] +Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, 5), ENSURE_REQUIREMENTS, [id=#90] (68) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, d_year#88, sum#90] -Keys [8]: [c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81, d_year#88] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#84 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#83 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#84 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#83 as decimal(8,2)))), DecimalType(8,2), true)))#92] -Results [2]: [c_customer_id#75 AS customer_id#93, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#84 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#83 as decimal(8,2)))), DecimalType(8,2), true)))#92,18,2) AS year_total#94] +Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91] +Results [2]: [c_customer_id#74 AS customer_id#92, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91,18,2) AS year_total#93] (69) BroadcastExchange -Input [2]: [customer_id#93, year_total#94] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#95] +Input [2]: [customer_id#92, year_total#93] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#94] (70) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#21] -Right keys [1]: [customer_id#93] -Join condition: (CASE WHEN (year_total#71 > 0.00) THEN CheckOverflow((promote_precision(year_total#94) / promote_precision(year_total#71)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#73) +Right keys [1]: [customer_id#92] +Join condition: (CASE WHEN (year_total#71 > 0.00) THEN CheckOverflow((promote_precision(year_total#93) / promote_precision(year_total#71)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#22 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#22)), DecimalType(38,20), true) ELSE 0E-20 END) (71) Project [codegen id : 16] Output [4]: [customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46] -Input [9]: [customer_id#21, customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#71, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#73, customer_id#93, year_total#94] +Input [10]: [customer_id#21, year_total#22, customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#47, year_total#71, customer_id#92, year_total#93] (72) TakeOrderedAndProject Input [4]: [customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46] @@ -437,7 +437,7 @@ Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2001)) AND isnotnull(d_date_ (76) BroadcastExchange Input [2]: [d_date_sk#15, d_year#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#34 IN dynamicpruning#35 BroadcastExchange (80) @@ -462,10 +462,10 @@ Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_ (80) BroadcastExchange Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#85 IN dynamicpruning#35 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#35 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index b313db49b564e..6ebf857eba13a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -1,8 +1,8 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] WholeStageCodegen (16) Project [customer_id,customer_first_name,customer_last_name,customer_email_address] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END] - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt index bf54f30085860..c8cd870c62c8a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt @@ -91,411 +91,345 @@ Right keys [1]: [ca_zip#9] Join condition: None (10) Project [codegen id : 2] -Output [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, upper(ca_country#10) AS upper(spark_catalog.default.customer_address.ca_country)#11] +Output [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] Input [8]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5, ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] (11) BroadcastExchange -Input [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, upper(spark_catalog.default.customer_address.ca_country)#11] -Arguments: HashedRelationBroadcastMode(List(input[3, int, true], input[5, string, true]),false), [id=#12] +Input [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] +Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#11] (12) Scan parquet default.customer -Output [5]: [c_customer_sk#13, c_current_addr_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Output [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (13) ColumnarToRow -Input [5]: [c_customer_sk#13, c_current_addr_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] (14) Filter -Input [5]: [c_customer_sk#13, c_current_addr_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : ((isnotnull(c_customer_sk#13) AND isnotnull(c_current_addr_sk#14)) AND isnotnull(c_birth_country#17)) +Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Condition : ((isnotnull(c_customer_sk#12) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_birth_country#16)) (15) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ca_address_sk#7, upper(spark_catalog.default.customer_address.ca_country)#11] -Right keys [2]: [c_current_addr_sk#14, c_birth_country#17] +Left keys [2]: [ca_address_sk#7, upper(ca_country#10)] +Right keys [2]: [c_current_addr_sk#13, c_birth_country#16] Join condition: None (16) Project [codegen id : 3] -Output [7]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#13, c_first_name#15, c_last_name#16] -Input [11]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, upper(spark_catalog.default.customer_address.ca_country)#11, c_customer_sk#13, c_current_addr_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Output [7]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#12, c_first_name#14, c_last_name#15] +Input [11]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10, c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] (17) BroadcastExchange -Input [7]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#13, c_first_name#15, c_last_name#16] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#18] +Input [7]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#12, c_first_name#14, c_last_name#15] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#17] (18) Scan parquet default.store_sales -Output [6]: [ss_item_sk#19, ss_customer_sk#20, ss_store_sk#21, ss_ticket_number#22, ss_net_paid#23, ss_sold_date_sk#24] +Output [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (19) ColumnarToRow -Input [6]: [ss_item_sk#19, ss_customer_sk#20, ss_store_sk#21, ss_ticket_number#22, ss_net_paid#23, ss_sold_date_sk#24] +Input [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] (20) Filter -Input [6]: [ss_item_sk#19, ss_customer_sk#20, ss_store_sk#21, ss_ticket_number#22, ss_net_paid#23, ss_sold_date_sk#24] -Condition : (((isnotnull(ss_ticket_number#22) AND isnotnull(ss_item_sk#19)) AND isnotnull(ss_store_sk#21)) AND isnotnull(ss_customer_sk#20)) +Input [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] +Condition : (((isnotnull(ss_ticket_number#21) AND isnotnull(ss_item_sk#18)) AND isnotnull(ss_store_sk#20)) AND isnotnull(ss_customer_sk#19)) (21) Project -Output [5]: [ss_item_sk#19, ss_customer_sk#20, ss_store_sk#21, ss_ticket_number#22, ss_net_paid#23] -Input [6]: [ss_item_sk#19, ss_customer_sk#20, ss_store_sk#21, ss_ticket_number#22, ss_net_paid#23, ss_sold_date_sk#24] +Output [5]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22] +Input [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] (22) BroadcastHashJoin [codegen id : 5] -Left keys [2]: [s_store_sk#1, c_customer_sk#13] -Right keys [2]: [ss_store_sk#21, ss_customer_sk#20] +Left keys [2]: [s_store_sk#1, c_customer_sk#12] +Right keys [2]: [ss_store_sk#20, ss_customer_sk#19] Join condition: None (23) Project [codegen id : 5] -Output [8]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_ticket_number#22, ss_net_paid#23] -Input [12]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#13, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_customer_sk#20, ss_store_sk#21, ss_ticket_number#22, ss_net_paid#23] +Output [8]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22] +Input [12]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#12, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22] (24) Scan parquet default.item -Output [6]: [i_item_sk#25, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] +Output [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#25, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] +Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] (26) Filter [codegen id : 4] -Input [6]: [i_item_sk#25, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] -Condition : ((isnotnull(i_color#28) AND (i_color#28 = pale )) AND isnotnull(i_item_sk#25)) +Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Condition : ((isnotnull(i_color#27) AND (i_color#27 = pale )) AND isnotnull(i_item_sk#24)) (27) BroadcastExchange -Input [6]: [i_item_sk#25, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#19] -Right keys [1]: [i_item_sk#25] +Left keys [1]: [ss_item_sk#18] +Right keys [1]: [i_item_sk#24] Join condition: None (29) Project [codegen id : 5] -Output [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_ticket_number#22, ss_net_paid#23, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] -Input [14]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_ticket_number#22, ss_net_paid#23, i_item_sk#25, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] +Output [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Input [14]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] (30) Exchange -Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_ticket_number#22, ss_net_paid#23, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] -Arguments: hashpartitioning(ss_ticket_number#22, ss_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Arguments: hashpartitioning(ss_ticket_number#21, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#31] (31) Sort [codegen id : 6] -Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_ticket_number#22, ss_net_paid#23, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30] -Arguments: [ss_ticket_number#22 ASC NULLS FIRST, ss_item_sk#19 ASC NULLS FIRST], false, 0 +Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Arguments: [ss_ticket_number#21 ASC NULLS FIRST, ss_item_sk#18 ASC NULLS FIRST], false, 0 (32) Scan parquet default.store_returns -Output [3]: [sr_item_sk#33, sr_ticket_number#34, sr_returned_date_sk#35] +Output [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (33) ColumnarToRow [codegen id : 7] -Input [3]: [sr_item_sk#33, sr_ticket_number#34, sr_returned_date_sk#35] +Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] (34) Filter [codegen id : 7] -Input [3]: [sr_item_sk#33, sr_ticket_number#34, sr_returned_date_sk#35] -Condition : (isnotnull(sr_ticket_number#34) AND isnotnull(sr_item_sk#33)) +Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) (35) Project [codegen id : 7] -Output [2]: [sr_item_sk#33, sr_ticket_number#34] -Input [3]: [sr_item_sk#33, sr_ticket_number#34, sr_returned_date_sk#35] +Output [2]: [sr_item_sk#32, sr_ticket_number#33] +Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] (36) Exchange -Input [2]: [sr_item_sk#33, sr_ticket_number#34] -Arguments: hashpartitioning(sr_ticket_number#34, sr_item_sk#33, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [2]: [sr_item_sk#32, sr_ticket_number#33] +Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#35] (37) Sort [codegen id : 8] -Input [2]: [sr_item_sk#33, sr_ticket_number#34] -Arguments: [sr_ticket_number#34 ASC NULLS FIRST, sr_item_sk#33 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#32, sr_ticket_number#33] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#22, ss_item_sk#19] -Right keys [2]: [sr_ticket_number#34, sr_item_sk#33] +Left keys [2]: [ss_ticket_number#21, ss_item_sk#18] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (39) Project [codegen id : 9] -Output [11]: [ss_net_paid#23, s_store_name#2, s_state#4, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30, c_first_name#15, c_last_name#16, ca_state#8] -Input [15]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#15, c_last_name#16, ss_item_sk#19, ss_ticket_number#22, ss_net_paid#23, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30, sr_item_sk#33, sr_ticket_number#34] +Output [11]: [ss_net_paid#22, s_store_name#2, s_state#4, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, c_first_name#14, c_last_name#15, ca_state#8] +Input [15]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, sr_item_sk#32, sr_ticket_number#33] (40) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#23, s_store_name#2, s_state#4, i_current_price#26, i_size#27, i_color#28, i_units#29, i_manager_id#30, c_first_name#15, c_last_name#16, ca_state#8] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#2, ca_state#8, s_state#4, i_color#28, i_current_price#26, i_manager_id#30, i_units#29, i_size#27] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#23))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#16, c_first_name#15, s_store_name#2, ca_state#8, s_state#4, i_color#28, i_current_price#26, i_manager_id#30, i_units#29, i_size#27, sum#38] +Input [11]: [ss_net_paid#22, s_store_name#2, s_state#4, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, c_first_name#14, c_last_name#15, ca_state#8] +Keys [10]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum#36] +Results [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#37] (41) Exchange -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#2, ca_state#8, s_state#4, i_color#28, i_current_price#26, i_manager_id#30, i_units#29, i_size#27, sum#38] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#2, ca_state#8, s_state#4, i_color#28, i_current_price#26, i_manager_id#30, i_units#29, i_size#27, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#37] +Arguments: hashpartitioning(c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, 5), ENSURE_REQUIREMENTS, [id=#38] (42) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#2, ca_state#8, s_state#4, i_color#28, i_current_price#26, i_manager_id#30, i_units#29, i_size#27, sum#38] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#2, ca_state#8, s_state#4, i_color#28, i_current_price#26, i_manager_id#30, i_units#29, i_size#27] -Functions [1]: [sum(UnscaledValue(ss_net_paid#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#23))#40] -Results [4]: [c_last_name#16, c_first_name#15, s_store_name#2, MakeDecimal(sum(UnscaledValue(ss_net_paid#23))#40,17,2) AS netpaid#41] +Input [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#37] +Keys [10]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26] +Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#39] +Results [4]: [c_last_name#15, c_first_name#14, s_store_name#2, MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#39,17,2) AS netpaid#40] (43) HashAggregate [codegen id : 10] -Input [4]: [c_last_name#16, c_first_name#15, s_store_name#2, netpaid#41] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#2] -Functions [1]: [partial_sum(netpaid#41)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [5]: [c_last_name#16, c_first_name#15, s_store_name#2, sum#44, isEmpty#45] +Input [4]: [c_last_name#15, c_first_name#14, s_store_name#2, netpaid#40] +Keys [3]: [c_last_name#15, c_first_name#14, s_store_name#2] +Functions [1]: [partial_sum(netpaid#40)] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [5]: [c_last_name#15, c_first_name#14, s_store_name#2, sum#43, isEmpty#44] (44) Exchange -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#2, sum#44, isEmpty#45] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#2, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [5]: [c_last_name#15, c_first_name#14, s_store_name#2, sum#43, isEmpty#44] +Arguments: hashpartitioning(c_last_name#15, c_first_name#14, s_store_name#2, 5), ENSURE_REQUIREMENTS, [id=#45] (45) HashAggregate [codegen id : 11] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#2, sum#44, isEmpty#45] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#2] -Functions [1]: [sum(netpaid#41)] -Aggregate Attributes [1]: [sum(netpaid#41)#47] -Results [4]: [c_last_name#16, c_first_name#15, s_store_name#2, sum(netpaid#41)#47 AS paid#48] +Input [5]: [c_last_name#15, c_first_name#14, s_store_name#2, sum#43, isEmpty#44] +Keys [3]: [c_last_name#15, c_first_name#14, s_store_name#2] +Functions [1]: [sum(netpaid#40)] +Aggregate Attributes [1]: [sum(netpaid#40)#46] +Results [4]: [c_last_name#15, c_first_name#14, s_store_name#2, sum(netpaid#40)#46 AS paid#47] (46) Filter [codegen id : 11] -Input [4]: [c_last_name#16, c_first_name#15, s_store_name#2, paid#48] -Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) +Input [4]: [c_last_name#15, c_first_name#14, s_store_name#2, paid#47] +Condition : (isnotnull(paid#47) AND (cast(paid#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) (47) Exchange -Input [4]: [c_last_name#16, c_first_name#15, s_store_name#2, paid#48] -Arguments: rangepartitioning(c_last_name#16 ASC NULLS FIRST, c_first_name#15 ASC NULLS FIRST, s_store_name#2 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [c_last_name#15, c_first_name#14, s_store_name#2, paid#47] +Arguments: rangepartitioning(c_last_name#15 ASC NULLS FIRST, c_first_name#14 ASC NULLS FIRST, s_store_name#2 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#50] (48) Sort [codegen id : 12] -Input [4]: [c_last_name#16, c_first_name#15, s_store_name#2, paid#48] -Arguments: [c_last_name#16 ASC NULLS FIRST, c_first_name#15 ASC NULLS FIRST, s_store_name#2 ASC NULLS FIRST], true, 0 +Input [4]: [c_last_name#15, c_first_name#14, s_store_name#2, paid#47] +Arguments: [c_last_name#15 ASC NULLS FIRST, c_first_name#14 ASC NULLS FIRST, s_store_name#2 ASC NULLS FIRST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (88) -+- Exchange (87) - +- * HashAggregate (86) - +- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- * Project (82) - +- * SortMergeJoin Inner (81) - :- * Sort (78) - : +- Exchange (77) - : +- * Project (76) - : +- * SortMergeJoin Inner (75) - : :- * Sort (69) - : : +- Exchange (68) - : : +- * Project (67) - : : +- * BroadcastHashJoin Inner BuildLeft (66) - : : :- BroadcastExchange (61) - : : : +- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildLeft (59) - : : : :- BroadcastExchange (55) - : : : : +- * Project (54) - : : : : +- * BroadcastHashJoin Inner BuildLeft (53) - : : : : :- ReusedExchange (49) - : : : : +- * Filter (52) - : : : : +- * ColumnarToRow (51) - : : : : +- Scan parquet default.customer_address (50) - : : : +- * Filter (58) - : : : +- * ColumnarToRow (57) - : : : +- Scan parquet default.customer (56) - : : +- * Project (65) - : : +- * Filter (64) - : : +- * ColumnarToRow (63) - : : +- Scan parquet default.store_sales (62) - : +- * Sort (74) - : +- Exchange (73) - : +- * Filter (72) - : +- * ColumnarToRow (71) - : +- Scan parquet default.item (70) - +- * Sort (80) - +- ReusedExchange (79) - - -(49) ReusedExchange [Reuses operator id: 5] -Output [4]: [s_store_sk#52, s_store_name#53, s_state#54, s_zip#55] - -(50) Scan parquet default.customer_address -Output [4]: [ca_address_sk#56, ca_state#57, ca_zip#58, ca_country#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] -ReadSchema: struct - -(51) ColumnarToRow -Input [4]: [ca_address_sk#56, ca_state#57, ca_zip#58, ca_country#59] - -(52) Filter -Input [4]: [ca_address_sk#56, ca_state#57, ca_zip#58, ca_country#59] -Condition : (((isnotnull(ca_address_sk#56) AND isnotnull(ca_country#59)) AND isnotnull(ca_zip#58)) AND isnotnull(upper(ca_country#59))) - -(53) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_zip#55] -Right keys [1]: [ca_zip#58] -Join condition: None - -(54) Project [codegen id : 2] -Output [6]: [s_store_sk#52, s_store_name#53, s_state#54, ca_address_sk#56, ca_state#57, upper(ca_country#59) AS upper(spark_catalog.default.customer_address.ca_country)#60] -Input [8]: [s_store_sk#52, s_store_name#53, s_state#54, s_zip#55, ca_address_sk#56, ca_state#57, ca_zip#58, ca_country#59] - -(55) BroadcastExchange -Input [6]: [s_store_sk#52, s_store_name#53, s_state#54, ca_address_sk#56, ca_state#57, upper(spark_catalog.default.customer_address.ca_country)#60] -Arguments: HashedRelationBroadcastMode(List(input[3, int, true], input[5, string, true]),false), [id=#61] - -(56) Scan parquet default.customer -Output [5]: [c_customer_sk#62, c_current_addr_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(57) ColumnarToRow -Input [5]: [c_customer_sk#62, c_current_addr_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] - -(58) Filter -Input [5]: [c_customer_sk#62, c_current_addr_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Condition : ((isnotnull(c_customer_sk#62) AND isnotnull(c_current_addr_sk#63)) AND isnotnull(c_birth_country#66)) - -(59) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ca_address_sk#56, upper(spark_catalog.default.customer_address.ca_country)#60] -Right keys [2]: [c_current_addr_sk#63, c_birth_country#66] -Join condition: None - -(60) Project [codegen id : 3] -Output [7]: [s_store_sk#52, s_store_name#53, s_state#54, ca_state#57, c_customer_sk#62, c_first_name#64, c_last_name#65] -Input [11]: [s_store_sk#52, s_store_name#53, s_state#54, ca_address_sk#56, ca_state#57, upper(spark_catalog.default.customer_address.ca_country)#60, c_customer_sk#62, c_current_addr_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] - -(61) BroadcastExchange -Input [7]: [s_store_sk#52, s_store_name#53, s_state#54, ca_state#57, c_customer_sk#62, c_first_name#64, c_last_name#65] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#67] - -(62) Scan parquet default.store_sales -Output [6]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72, ss_sold_date_sk#73] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* HashAggregate (76) ++- Exchange (75) + +- * HashAggregate (74) + +- * HashAggregate (73) + +- Exchange (72) + +- * HashAggregate (71) + +- * Project (70) + +- * SortMergeJoin Inner (69) + :- * Sort (66) + : +- Exchange (65) + : +- * Project (64) + : +- * SortMergeJoin Inner (63) + : :- * Sort (57) + : : +- Exchange (56) + : : +- * Project (55) + : : +- * BroadcastHashJoin Inner BuildLeft (54) + : : :- ReusedExchange (49) + : : +- * Project (53) + : : +- * Filter (52) + : : +- * ColumnarToRow (51) + : : +- Scan parquet default.store_sales (50) + : +- * Sort (62) + : +- Exchange (61) + : +- * Filter (60) + : +- * ColumnarToRow (59) + : +- Scan parquet default.item (58) + +- * Sort (68) + +- ReusedExchange (67) + + +(49) ReusedExchange [Reuses operator id: 17] +Output [7]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#54, c_customer_sk#55, c_first_name#56, c_last_name#57] + +(50) Scan parquet default.store_sales +Output [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(63) ColumnarToRow -Input [6]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72, ss_sold_date_sk#73] +(51) ColumnarToRow +Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] -(64) Filter -Input [6]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72, ss_sold_date_sk#73] -Condition : (((isnotnull(ss_ticket_number#71) AND isnotnull(ss_item_sk#68)) AND isnotnull(ss_store_sk#70)) AND isnotnull(ss_customer_sk#69)) +(52) Filter +Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] +Condition : (((isnotnull(ss_ticket_number#61) AND isnotnull(ss_item_sk#58)) AND isnotnull(ss_store_sk#60)) AND isnotnull(ss_customer_sk#59)) -(65) Project -Output [5]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72] -Input [6]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72, ss_sold_date_sk#73] +(53) Project +Output [5]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62] +Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] -(66) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [s_store_sk#52, c_customer_sk#62] -Right keys [2]: [ss_store_sk#70, ss_customer_sk#69] +(54) BroadcastHashJoin [codegen id : 4] +Left keys [2]: [s_store_sk#51, c_customer_sk#55] +Right keys [2]: [ss_store_sk#60, ss_customer_sk#59] Join condition: None -(67) Project [codegen id : 4] -Output [8]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72] -Input [12]: [s_store_sk#52, s_store_name#53, s_state#54, ca_state#57, c_customer_sk#62, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72] +(55) Project [codegen id : 4] +Output [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] +Input [12]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#54, c_customer_sk#55, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62] -(68) Exchange -Input [8]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72] -Arguments: hashpartitioning(ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#74] +(56) Exchange +Input [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] +Arguments: hashpartitioning(ss_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#64] -(69) Sort [codegen id : 5] -Input [8]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72] -Arguments: [ss_item_sk#68 ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 5] +Input [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] +Arguments: [ss_item_sk#58 ASC NULLS FIRST], false, 0 -(70) Scan parquet default.item -Output [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +(58) Scan parquet default.item +Output [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(71) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +(59) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -(72) Filter [codegen id : 6] -Input [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] -Condition : isnotnull(i_item_sk#75) +(60) Filter [codegen id : 6] +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Condition : isnotnull(i_item_sk#65) -(73) Exchange -Input [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] -Arguments: hashpartitioning(i_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] +(61) Exchange +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Arguments: hashpartitioning(i_item_sk#65, 5), ENSURE_REQUIREMENTS, [id=#71] -(74) Sort [codegen id : 7] -Input [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] -Arguments: [i_item_sk#75 ASC NULLS FIRST], false, 0 +(62) Sort [codegen id : 7] +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Arguments: [i_item_sk#65 ASC NULLS FIRST], false, 0 -(75) SortMergeJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#68] -Right keys [1]: [i_item_sk#75] +(63) SortMergeJoin [codegen id : 8] +Left keys [1]: [ss_item_sk#58] +Right keys [1]: [i_item_sk#65] Join condition: None -(76) Project [codegen id : 8] -Output [13]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] -Input [14]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +(64) Project [codegen id : 8] +Output [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Input [14]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -(77) Exchange -Input [13]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] -Arguments: hashpartitioning(ss_ticket_number#71, ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#82] +(65) Exchange +Input [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Arguments: hashpartitioning(ss_ticket_number#61, ss_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#72] -(78) Sort [codegen id : 9] -Input [13]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] -Arguments: [ss_ticket_number#71 ASC NULLS FIRST, ss_item_sk#68 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 9] +Input [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Arguments: [ss_ticket_number#61 ASC NULLS FIRST, ss_item_sk#58 ASC NULLS FIRST], false, 0 -(79) ReusedExchange [Reuses operator id: 36] -Output [2]: [sr_item_sk#83, sr_ticket_number#84] +(67) ReusedExchange [Reuses operator id: 36] +Output [2]: [sr_item_sk#73, sr_ticket_number#74] -(80) Sort [codegen id : 11] -Input [2]: [sr_item_sk#83, sr_ticket_number#84] -Arguments: [sr_ticket_number#84 ASC NULLS FIRST, sr_item_sk#83 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 11] +Input [2]: [sr_item_sk#73, sr_ticket_number#74] +Arguments: [sr_ticket_number#74 ASC NULLS FIRST, sr_item_sk#73 ASC NULLS FIRST], false, 0 -(81) SortMergeJoin [codegen id : 12] -Left keys [2]: [ss_ticket_number#71, ss_item_sk#68] -Right keys [2]: [sr_ticket_number#84, sr_item_sk#83] +(69) SortMergeJoin [codegen id : 12] +Left keys [2]: [ss_ticket_number#61, ss_item_sk#58] +Right keys [2]: [sr_ticket_number#74, sr_item_sk#73] Join condition: None -(82) Project [codegen id : 12] -Output [11]: [ss_net_paid#72, s_store_name#53, s_state#54, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80, c_first_name#64, c_last_name#65, ca_state#57] -Input [15]: [s_store_name#53, s_state#54, ca_state#57, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80, sr_item_sk#83, sr_ticket_number#84] - -(83) HashAggregate [codegen id : 12] -Input [11]: [ss_net_paid#72, s_store_name#53, s_state#54, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80, c_first_name#64, c_last_name#65, ca_state#57] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#53, ca_state#57, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#72))] -Aggregate Attributes [1]: [sum#85] -Results [11]: [c_last_name#65, c_first_name#64, s_store_name#53, ca_state#57, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77, sum#86] - -(84) Exchange -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#53, ca_state#57, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77, sum#86] -Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#53, ca_state#57, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77, 5), ENSURE_REQUIREMENTS, [id=#87] - -(85) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#53, ca_state#57, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77, sum#86] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#53, ca_state#57, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77] -Functions [1]: [sum(UnscaledValue(ss_net_paid#72))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#72))#88] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#72))#88,17,2) AS netpaid#41] - -(86) HashAggregate [codegen id : 13] -Input [1]: [netpaid#41] +(70) Project [codegen id : 12] +Output [11]: [ss_net_paid#62, s_store_name#52, s_state#53, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#56, c_last_name#57, ca_state#54] +Input [15]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, sr_item_sk#73, sr_ticket_number#74] + +(71) HashAggregate [codegen id : 12] +Input [11]: [ss_net_paid#62, s_store_name#52, s_state#53, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#56, c_last_name#57, ca_state#54] +Keys [10]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#62))] +Aggregate Attributes [1]: [sum#75] +Results [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] + +(72) Exchange +Input [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] +Arguments: hashpartitioning(c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, 5), ENSURE_REQUIREMENTS, [id=#77] + +(73) HashAggregate [codegen id : 13] +Input [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] +Keys [10]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67] +Functions [1]: [sum(UnscaledValue(ss_net_paid#62))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#62))#78] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#62))#78,17,2) AS netpaid#40] + +(74) HashAggregate [codegen id : 13] +Input [1]: [netpaid#40] Keys: [] -Functions [1]: [partial_avg(netpaid#41)] -Aggregate Attributes [2]: [sum#89, count#90] -Results [2]: [sum#91, count#92] +Functions [1]: [partial_avg(netpaid#40)] +Aggregate Attributes [2]: [sum#79, count#80] +Results [2]: [sum#81, count#82] -(87) Exchange -Input [2]: [sum#91, count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] +(75) Exchange +Input [2]: [sum#81, count#82] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#83] -(88) HashAggregate [codegen id : 14] -Input [2]: [sum#91, count#92] +(76) HashAggregate [codegen id : 14] +Input [2]: [sum#81, count#82] Keys: [] -Functions [1]: [avg(netpaid#41)] -Aggregate Attributes [1]: [avg(netpaid#41)#94] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#41)#94)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#95] +Functions [1]: [avg(netpaid#40)] +Aggregate Attributes [1]: [avg(netpaid#40)#84] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#84)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#85] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt index bdc45d61509bf..4beebcbbe52ef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt @@ -35,25 +35,7 @@ WholeStageCodegen (12) Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid] BroadcastHashJoin [s_store_sk,c_customer_sk,ss_store_sk,ss_customer_sk] InputAdapter - BroadcastExchange #14 - WholeStageCodegen (3) - Project [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] - BroadcastHashJoin [ca_address_sk,upper(spark_catalog.default.customer_address.ca_country),c_current_addr_sk,c_birth_country] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (2) - Project [s_store_sk,s_store_name,s_state,ca_address_sk,ca_state,ca_country] - BroadcastHashJoin [s_zip,ca_zip] - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #7 - Filter [ca_address_sk,ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] - Filter [c_customer_sk,c_current_addr_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + ReusedExchange [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] #5 Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow @@ -63,7 +45,7 @@ WholeStageCodegen (12) WholeStageCodegen (7) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #16 + Exchange [i_item_sk] #14 WholeStageCodegen (6) Filter [i_item_sk] ColumnarToRow @@ -100,7 +82,7 @@ WholeStageCodegen (12) BroadcastExchange #5 WholeStageCodegen (3) Project [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] - BroadcastHashJoin [ca_address_sk,upper(spark_catalog.default.customer_address.ca_country),c_current_addr_sk,c_birth_country] + BroadcastHashJoin [ca_address_sk,ca_country,c_current_addr_sk,c_birth_country] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt index b205e0f39e8b2..ef15af68693a3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt @@ -1,15 +1,15 @@ == Physical Plan == -* Sort (49) -+- Exchange (48) - +- * Filter (47) - +- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) +* Sort (48) ++- Exchange (47) + +- * Filter (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) :- * Project (33) : +- * BroadcastHashJoin Inner BuildRight (32) : :- * Project (27) @@ -43,11 +43,10 @@ : +- * Filter (30) : +- * ColumnarToRow (29) : +- Scan parquet default.customer (28) - +- BroadcastExchange (38) - +- * Project (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.customer_address (34) + +- BroadcastExchange (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.customer_address (34) (1) Scan parquet default.store_sales @@ -210,246 +209,219 @@ Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] Condition : ((isnotnull(ca_address_sk#31) AND isnotnull(ca_country#34)) AND isnotnull(ca_zip#33)) -(37) Project [codegen id : 8] -Output [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, upper(ca_country#34) AS upper(spark_catalog.default.customer_address.ca_country)#35] +(37) BroadcastExchange Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#35] -(38) BroadcastExchange -Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, upper(spark_catalog.default.customer_address.ca_country)#35] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[3, string, true], input[2, string, true]),false), [id=#36] - -(39) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 9] Left keys [3]: [c_current_addr_sk#26, c_birth_country#29, s_zip#16] -Right keys [3]: [ca_address_sk#31, upper(spark_catalog.default.customer_address.ca_country)#35, ca_zip#33] +Right keys [3]: [ca_address_sk#31, upper(ca_country#34), ca_zip#33] Join condition: None -(40) Project [codegen id : 9] +(39) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#27, c_last_name#28, ca_state#32] -Input [17]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29, ca_address_sk#31, ca_state#32, ca_zip#33, upper(spark_catalog.default.customer_address.ca_country)#35] +Input [17]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29, ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] -(41) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#27, c_last_name#28, ca_state#32] Keys [10]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#38] +Aggregate Attributes [1]: [sum#36] +Results [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#37] -(42) Exchange -Input [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#38] -Arguments: hashpartitioning(c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#39] +(41) Exchange +Input [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#37] +Arguments: hashpartitioning(c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#38] -(43) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#38] +(42) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#37] Keys [10]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#40] -Results [4]: [c_last_name#28, c_first_name#27, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#40,17,2) AS netpaid#41] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] +Results [4]: [c_last_name#28, c_first_name#27, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(44) HashAggregate [codegen id : 10] -Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, netpaid#41] +(43) HashAggregate [codegen id : 10] +Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, netpaid#40] Keys [3]: [c_last_name#28, c_first_name#27, s_store_name#13] -Functions [1]: [partial_sum(netpaid#41)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum#44, isEmpty#45] +Functions [1]: [partial_sum(netpaid#40)] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum#43, isEmpty#44] -(45) Exchange -Input [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum#44, isEmpty#45] -Arguments: hashpartitioning(c_last_name#28, c_first_name#27, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#46] +(44) Exchange +Input [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum#43, isEmpty#44] +Arguments: hashpartitioning(c_last_name#28, c_first_name#27, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#45] -(46) HashAggregate [codegen id : 11] -Input [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum#44, isEmpty#45] +(45) HashAggregate [codegen id : 11] +Input [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum#43, isEmpty#44] Keys [3]: [c_last_name#28, c_first_name#27, s_store_name#13] -Functions [1]: [sum(netpaid#41)] -Aggregate Attributes [1]: [sum(netpaid#41)#47] -Results [4]: [c_last_name#28, c_first_name#27, s_store_name#13, sum(netpaid#41)#47 AS paid#48] +Functions [1]: [sum(netpaid#40)] +Aggregate Attributes [1]: [sum(netpaid#40)#46] +Results [4]: [c_last_name#28, c_first_name#27, s_store_name#13, sum(netpaid#40)#46 AS paid#47] -(47) Filter [codegen id : 11] -Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#48] -Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) +(46) Filter [codegen id : 11] +Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#47] +Condition : (isnotnull(paid#47) AND (cast(paid#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) -(48) Exchange -Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#48] -Arguments: rangepartitioning(c_last_name#28 ASC NULLS FIRST, c_first_name#27 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#51] +(47) Exchange +Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#47] +Arguments: rangepartitioning(c_last_name#28 ASC NULLS FIRST, c_first_name#27 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#50] -(49) Sort [codegen id : 12] -Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#48] +(48) Sort [codegen id : 12] +Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#47] Arguments: [c_last_name#28 ASC NULLS FIRST, c_first_name#27 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (80) -+- Exchange (79) - +- * HashAggregate (78) - +- * HashAggregate (77) - +- Exchange (76) - +- * HashAggregate (75) - +- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (58) - : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : :- * Project (55) - : : : : +- * SortMergeJoin Inner (54) - : : : : :- * Sort (51) - : : : : : +- ReusedExchange (50) - : : : : +- * Sort (53) - : : : : +- ReusedExchange (52) - : : : +- ReusedExchange (56) - : : +- BroadcastExchange (62) - : : +- * Filter (61) - : : +- * ColumnarToRow (60) - : : +- Scan parquet default.item (59) - : +- ReusedExchange (65) - +- BroadcastExchange (72) - +- * Project (71) - +- * Filter (70) - +- * ColumnarToRow (69) - +- Scan parquet default.customer_address (68) - - -(50) ReusedExchange [Reuses operator id: 5] -Output [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] - -(51) Sort [codegen id : 2] -Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] -Arguments: [ss_ticket_number#55 ASC NULLS FIRST, ss_item_sk#52 ASC NULLS FIRST], false, 0 - -(52) ReusedExchange [Reuses operator id: 11] -Output [2]: [sr_item_sk#57, sr_ticket_number#58] - -(53) Sort [codegen id : 4] -Input [2]: [sr_item_sk#57, sr_ticket_number#58] -Arguments: [sr_ticket_number#58 ASC NULLS FIRST, sr_item_sk#57 ASC NULLS FIRST], false, 0 - -(54) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#55, ss_item_sk#52] -Right keys [2]: [sr_ticket_number#58, sr_item_sk#57] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* HashAggregate (75) ++- Exchange (74) + +- * HashAggregate (73) + +- * HashAggregate (72) + +- Exchange (71) + +- * HashAggregate (70) + +- * Project (69) + +- * BroadcastHashJoin Inner BuildRight (68) + :- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * Project (57) + : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : :- * Project (54) + : : : : +- * SortMergeJoin Inner (53) + : : : : :- * Sort (50) + : : : : : +- ReusedExchange (49) + : : : : +- * Sort (52) + : : : : +- ReusedExchange (51) + : : : +- ReusedExchange (55) + : : +- BroadcastExchange (61) + : : +- * Filter (60) + : : +- * ColumnarToRow (59) + : : +- Scan parquet default.item (58) + : +- ReusedExchange (64) + +- ReusedExchange (67) + + +(49) ReusedExchange [Reuses operator id: 5] +Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] + +(50) Sort [codegen id : 2] +Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] +Arguments: [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST], false, 0 + +(51) ReusedExchange [Reuses operator id: 11] +Output [2]: [sr_item_sk#56, sr_ticket_number#57] + +(52) Sort [codegen id : 4] +Input [2]: [sr_item_sk#56, sr_ticket_number#57] +Arguments: [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST], false, 0 + +(53) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#54, ss_item_sk#51] +Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] Join condition: None -(55) Project [codegen id : 9] -Output [4]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56] -Input [7]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, sr_item_sk#57, sr_ticket_number#58] +(54) Project [codegen id : 9] +Output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] +Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] -(56) ReusedExchange [Reuses operator id: 19] -Output [4]: [s_store_sk#59, s_store_name#60, s_state#61, s_zip#62] +(55) ReusedExchange [Reuses operator id: 19] +Output [4]: [s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] -(57) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#59] +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#53] +Right keys [1]: [s_store_sk#58] Join condition: None -(58) Project [codegen id : 9] -Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#60, s_state#61, s_zip#62] -Input [8]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56, s_store_sk#59, s_store_name#60, s_state#61, s_zip#62] +(57) Project [codegen id : 9] +Output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61] +Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] -(59) Scan parquet default.item -Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +(58) Scan parquet default.item +Output [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(60) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +(59) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -(61) Filter [codegen id : 6] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Condition : isnotnull(i_item_sk#63) +(60) Filter [codegen id : 6] +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Condition : isnotnull(i_item_sk#62) -(62) BroadcastExchange -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] +(61) BroadcastExchange +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] -(63) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#52] -Right keys [1]: [i_item_sk#63] +(62) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#51] +Right keys [1]: [i_item_sk#62] Join condition: None -(64) Project [codegen id : 9] -Output [10]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#60, s_state#61, s_zip#62, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Input [12]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#60, s_state#61, s_zip#62, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +(63) Project [codegen id : 9] +Output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -(65) ReusedExchange [Reuses operator id: 31] -Output [5]: [c_customer_sk#70, c_current_addr_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +(64) ReusedExchange [Reuses operator id: 31] +Output [5]: [c_customer_sk#69, c_current_addr_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] -(66) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#53] -Right keys [1]: [c_customer_sk#70] +(65) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#52] +Right keys [1]: [c_customer_sk#69] Join condition: None -(67) Project [codegen id : 9] -Output [13]: [ss_net_paid#56, s_store_name#60, s_state#61, s_zip#62, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_current_addr_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -Input [15]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#60, s_state#61, s_zip#62, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#70, c_current_addr_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] - -(68) Scan parquet default.customer_address -Output [4]: [ca_address_sk#75, ca_state#76, ca_zip#77, ca_country#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] -ReadSchema: struct - -(69) ColumnarToRow [codegen id : 8] -Input [4]: [ca_address_sk#75, ca_state#76, ca_zip#77, ca_country#78] - -(70) Filter [codegen id : 8] -Input [4]: [ca_address_sk#75, ca_state#76, ca_zip#77, ca_country#78] -Condition : (((isnotnull(ca_address_sk#75) AND isnotnull(ca_country#78)) AND isnotnull(ca_zip#77)) AND isnotnull(upper(ca_country#78))) - -(71) Project [codegen id : 8] -Output [4]: [ca_address_sk#75, ca_state#76, ca_zip#77, upper(ca_country#78) AS upper(spark_catalog.default.customer_address.ca_country)#79] -Input [4]: [ca_address_sk#75, ca_state#76, ca_zip#77, ca_country#78] +(66) Project [codegen id : 9] +Output [13]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_current_addr_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_customer_sk#69, c_current_addr_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] -(72) BroadcastExchange -Input [4]: [ca_address_sk#75, ca_state#76, ca_zip#77, upper(spark_catalog.default.customer_address.ca_country)#79] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[3, string, true], input[2, string, true]),false), [id=#80] +(67) ReusedExchange [Reuses operator id: 37] +Output [4]: [ca_address_sk#74, ca_state#75, ca_zip#76, ca_country#77] -(73) BroadcastHashJoin [codegen id : 9] -Left keys [3]: [c_current_addr_sk#71, c_birth_country#74, s_zip#62] -Right keys [3]: [ca_address_sk#75, upper(spark_catalog.default.customer_address.ca_country)#79, ca_zip#77] +(68) BroadcastHashJoin [codegen id : 9] +Left keys [3]: [c_current_addr_sk#70, c_birth_country#73, s_zip#61] +Right keys [3]: [ca_address_sk#74, upper(ca_country#77), ca_zip#76] Join condition: None -(74) Project [codegen id : 9] -Output [11]: [ss_net_paid#56, s_store_name#60, s_state#61, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#76] -Input [17]: [ss_net_paid#56, s_store_name#60, s_state#61, s_zip#62, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_current_addr_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74, ca_address_sk#75, ca_state#76, ca_zip#77, upper(spark_catalog.default.customer_address.ca_country)#79] - -(75) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#56, s_store_name#60, s_state#61, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#76] -Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#60, ca_state#76, s_state#61, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#56))] -Aggregate Attributes [1]: [sum#81] -Results [11]: [c_last_name#73, c_first_name#72, s_store_name#60, ca_state#76, s_state#61, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#82] - -(76) Exchange -Input [11]: [c_last_name#73, c_first_name#72, s_store_name#60, ca_state#76, s_state#61, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#82] -Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#60, ca_state#76, s_state#61, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#83] - -(77) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#73, c_first_name#72, s_store_name#60, ca_state#76, s_state#61, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#82] -Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#60, ca_state#76, s_state#61, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] -Functions [1]: [sum(UnscaledValue(ss_net_paid#56))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#56))#84] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#56))#84,17,2) AS netpaid#41] - -(78) HashAggregate [codegen id : 10] -Input [1]: [netpaid#41] +(69) Project [codegen id : 9] +Output [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#75] +Input [17]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_current_addr_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73, ca_address_sk#74, ca_state#75, ca_zip#76, ca_country#77] + +(70) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#75] +Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [1]: [sum#78] +Results [11]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#79] + +(71) Exchange +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#79] +Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#80] + +(72) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#79] +Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] +Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#81] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#81,17,2) AS netpaid#40] + +(73) HashAggregate [codegen id : 10] +Input [1]: [netpaid#40] Keys: [] -Functions [1]: [partial_avg(netpaid#41)] -Aggregate Attributes [2]: [sum#85, count#86] -Results [2]: [sum#87, count#88] +Functions [1]: [partial_avg(netpaid#40)] +Aggregate Attributes [2]: [sum#82, count#83] +Results [2]: [sum#84, count#85] -(79) Exchange -Input [2]: [sum#87, count#88] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#89] +(74) Exchange +Input [2]: [sum#84, count#85] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#86] -(80) HashAggregate [codegen id : 11] -Input [2]: [sum#87, count#88] +(75) HashAggregate [codegen id : 11] +Input [2]: [sum#84, count#85] Keys: [] -Functions [1]: [avg(netpaid#41)] -Aggregate Attributes [1]: [avg(netpaid#41)#90] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#41)#90)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#91] +Functions [1]: [avg(netpaid#40)] +Aggregate Attributes [1]: [avg(netpaid#40)#87] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#87)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#88] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt index d49112a3b65c0..0550ba1f05d58 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (12) WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,upper(spark_catalog.default.customer_address.ca_country),ca_zip] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -48,13 +48,7 @@ WholeStageCodegen (12) InputAdapter ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Project [ca_address_sk,ca_state,ca_zip,ca_country] - Filter [ca_address_sk,ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name,s_store_name] #2 @@ -66,7 +60,7 @@ WholeStageCodegen (12) WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,upper(spark_catalog.default.customer_address.ca_country),ca_zip] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -122,8 +116,7 @@ WholeStageCodegen (12) InputAdapter BroadcastExchange #9 WholeStageCodegen (8) - Project [ca_address_sk,ca_state,ca_zip,ca_country] - Filter [ca_address_sk,ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + Filter [ca_address_sk,ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] 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 15c7c9cac4a94..fcb93277bc76a 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 @@ -196,53 +196,53 @@ Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_yea Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#26, i_brand#27, s_store_name#28, s_company_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, (rn#37 + 1) AS (v1_lag.rn + 1)#38] +Output [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] Input [8]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] (35) BroadcastExchange -Input [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, (v1_lag.rn + 1)#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], input[5, int, false]),false), [id=#39] +Input [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [id=#38] (36) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#24] -Right keys [5]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, (v1_lag.rn + 1)#38] +Right keys [5]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, (rn#37 + 1)] Join condition: None (37) Project [codegen id : 22] Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [15]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Input [15]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] (38) ReusedExchange [Reuses operator id: 31] -Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] +Output [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] (39) Sort [codegen id : 20] -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, s_store_name#41 ASC NULLS FIRST, s_company_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 (40) Window -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] -Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#40, i_brand#41, s_store_name#42, s_company_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] +Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#39, i_brand#40, s_store_name#41, s_company_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, (rn#47 - 1) AS (v1_lead.rn - 1)#48] -Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] +Output [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] +Input [8]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] (42) BroadcastExchange -Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, (v1_lead.rn - 1)#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], input[5, int, false]),false), [id=#49] +Input [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [id=#47] (43) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#24] -Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (v1_lead.rn - 1)#48] +Right keys [5]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, (rn#46 - 1)] Join condition: None (44) Project [codegen id : 22] -Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#50, sum_sales#46 AS nsum#51] -Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, (v1_lead.rn - 1)#48] +Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#48, sum_sales#45 AS nsum#49] +Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] (45) TakeOrderedAndProject -Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#50, nsum#51] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST], [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#50, nsum#51] +Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#48, nsum#49] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST], [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#48, nsum#49] ===== Subqueries ===== @@ -269,6 +269,6 @@ Condition : ((((d_year#11 = 1999) OR ((d_year#11 = 1998) AND (d_moy#12 = 12))) O (49) BroadcastExchange Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index 4f2c0b0a78a06..003a906be0dd9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] WholeStageCodegen (22) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] Filter [avg_monthly_sales,sum_sales] InputAdapter 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 59ef1e89952e9..1fb68bac3c7d8 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 @@ -196,53 +196,53 @@ Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sale Arguments: [rank(d_year#28, d_moy#29) windowspecdefinition(i_category#25, i_brand#26, cc_name#27, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#25, i_brand#26, cc_name#27], [d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, (rn#35 + 1) AS (v1_lag.rn + 1)#36] +Output [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] Input [7]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#33, rn#35] (35) BroadcastExchange -Input [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, (v1_lag.rn + 1)#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[4, int, false]),false), [id=#37] +Input [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#36] (36) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#23] -Right keys [4]: [i_category#25, i_brand#26, cc_name#27, (v1_lag.rn + 1)#36] +Right keys [4]: [i_category#25, i_brand#26, cc_name#27, (rn#35 + 1)] Join condition: None (37) Project [codegen id : 22] Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33] -Input [13]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, i_category#25, i_brand#26, cc_name#27, sum_sales#33, (v1_lag.rn + 1)#36] +Input [13]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] (38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +Output [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] (39) Sort [codegen id : 20] -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] +Arguments: [i_category#37 ASC NULLS FIRST, i_brand#38 ASC NULLS FIRST, cc_name#39 ASC NULLS FIRST, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST], false, 0 (40) Window -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -Arguments: [rank(d_year#41, d_moy#42) windowspecdefinition(i_category#38, i_brand#39, cc_name#40, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] +Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] +Arguments: [rank(d_year#40, d_moy#41) windowspecdefinition(i_category#37, i_brand#38, cc_name#39, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#43], [i_category#37, i_brand#38, cc_name#39], [d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, (rn#44 - 1) AS (v1_lead.rn - 1)#45] -Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] +Output [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] +Input [7]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42, rn#43] (42) BroadcastExchange -Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, (v1_lead.rn - 1)#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[4, int, false]),false), [id=#46] +Input [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#44] (43) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#23] -Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (v1_lead.rn - 1)#45] +Right keys [4]: [i_category#37, i_brand#38, cc_name#39, (rn#43 - 1)] Join condition: None (44) Project [codegen id : 22] -Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, sum_sales#33 AS psum#47, sum_sales#43 AS nsum#48] -Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33, i_category#38, i_brand#39, cc_name#40, sum_sales#43, (v1_lead.rn - 1)#45] +Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, sum_sales#33 AS psum#45, sum_sales#42 AS nsum#46] +Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33, i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] (45) TakeOrderedAndProject -Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#47, nsum#48] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#11 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#47, nsum#48] +Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#45, nsum#46] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#11 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#45, nsum#46] ===== Subqueries ===== @@ -269,6 +269,6 @@ Condition : ((((d_year#11 = 1999) OR ((d_year#11 = 1998) AND (d_moy#12 = 12))) O (49) BroadcastExchange Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index 13080dde396c8..f099cef3b9d02 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] WholeStageCodegen (22) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lead.rn - 1)] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lag.rn + 1)] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] Filter [avg_monthly_sales,sum_sales] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt index 6f6786c9eee3f..e838025a71db8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Exchange [cs_item_sk,d_date_sk] #3 WholeStageCodegen (10) Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d1.d_date + INTERVAL '5 days'] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date,i_item_desc] SortMergeJoin [cs_item_sk,i_item_sk] InputAdapter @@ -43,8 +43,25 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (2) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #5 + BroadcastExchange #7 WholeStageCodegen (1) Project [hd_demo_sk] Filter [hd_buy_potential,hd_demo_sk] @@ -52,7 +69,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #6 + BroadcastExchange #8 WholeStageCodegen (2) Project [cd_demo_sk] Filter [cd_marital_status,cd_demo_sk] @@ -60,7 +77,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - BroadcastExchange #7 + BroadcastExchange #9 WholeStageCodegen (3) Filter [d_date,d_date_sk] ColumnarToRow @@ -70,29 +87,14 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom WholeStageCodegen (7) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #8 + Exchange [i_item_sk] #10 WholeStageCodegen (6) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [d_date_sk,d_week_seq,d_date_sk,d_date] - BroadcastHashJoin [d_week_seq,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #5 InputAdapter WholeStageCodegen (14) Sort [inv_item_sk,inv_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt index e684dba8c697c..7968b1cb84729 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d1.d_date + INTERVAL '5 days'] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt index a219c72cfe0d4..1554259f337c1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt @@ -320,93 +320,93 @@ Right keys [1]: [customer_id#54] Join condition: None (53) Project [codegen id : 16] -Output [6]: [customer_id#16, customer_id#33, customer_first_name#34, customer_last_name#35, year_total#55, CASE WHEN (year_total#17 > 0.00) THEN CheckOverflow((promote_precision(year_total#36) / promote_precision(year_total#17)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#57] +Output [7]: [customer_id#16, year_total#17, customer_id#33, customer_first_name#34, customer_last_name#35, year_total#36, year_total#55] Input [8]: [customer_id#16, year_total#17, customer_id#33, customer_first_name#34, customer_last_name#35, year_total#36, customer_id#54, year_total#55] (54) Scan parquet default.customer -Output [4]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61] +Output [4]: [c_customer_sk#57, c_customer_id#58, c_first_name#59, c_last_name#60] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (55) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61] +Input [4]: [c_customer_sk#57, c_customer_id#58, c_first_name#59, c_last_name#60] (56) Filter [codegen id : 14] -Input [4]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61] -Condition : (isnotnull(c_customer_sk#58) AND isnotnull(c_customer_id#59)) +Input [4]: [c_customer_sk#57, c_customer_id#58, c_first_name#59, c_last_name#60] +Condition : (isnotnull(c_customer_sk#57) AND isnotnull(c_customer_id#58)) (57) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] +Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#25)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] (59) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] -Condition : isnotnull(ws_bill_customer_sk#62) +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_customer_sk#61) (60) BroadcastExchange -Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] (61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#58] -Right keys [1]: [ws_bill_customer_sk#62] +Left keys [1]: [c_customer_sk#57] +Right keys [1]: [ws_bill_customer_sk#61] Join condition: None (62) Project [codegen id : 14] -Output [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, ws_sold_date_sk#64] -Input [7]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] +Output [5]: [c_customer_id#58, c_first_name#59, c_last_name#60, ws_net_paid#62, ws_sold_date_sk#63] +Input [7]: [c_customer_sk#57, c_customer_id#58, c_first_name#59, c_last_name#60, ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] (63) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#66, d_year#67] +Output [2]: [d_date_sk#65, d_year#66] (64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#64] -Right keys [1]: [d_date_sk#66] +Left keys [1]: [ws_sold_date_sk#63] +Right keys [1]: [d_date_sk#65] Join condition: None (65) Project [codegen id : 14] -Output [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, d_year#67] -Input [7]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, ws_sold_date_sk#64, d_date_sk#66, d_year#67] +Output [5]: [c_customer_id#58, c_first_name#59, c_last_name#60, ws_net_paid#62, d_year#66] +Input [7]: [c_customer_id#58, c_first_name#59, c_last_name#60, ws_net_paid#62, ws_sold_date_sk#63, d_date_sk#65, d_year#66] (66) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, d_year#67] -Keys [4]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#67] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#63))] -Aggregate Attributes [1]: [sum#68] -Results [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#67, sum#69] +Input [5]: [c_customer_id#58, c_first_name#59, c_last_name#60, ws_net_paid#62, d_year#66] +Keys [4]: [c_customer_id#58, c_first_name#59, c_last_name#60, d_year#66] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum#67] +Results [5]: [c_customer_id#58, c_first_name#59, c_last_name#60, d_year#66, sum#68] (67) Exchange -Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#67, sum#69] -Arguments: hashpartitioning(c_customer_id#59, c_first_name#60, c_last_name#61, d_year#67, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [5]: [c_customer_id#58, c_first_name#59, c_last_name#60, d_year#66, sum#68] +Arguments: hashpartitioning(c_customer_id#58, c_first_name#59, c_last_name#60, d_year#66, 5), ENSURE_REQUIREMENTS, [id=#69] (68) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#67, sum#69] -Keys [4]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#67] -Functions [1]: [sum(UnscaledValue(ws_net_paid#63))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#63))#71] -Results [2]: [c_customer_id#59 AS customer_id#72, MakeDecimal(sum(UnscaledValue(ws_net_paid#63))#71,17,2) AS year_total#73] +Input [5]: [c_customer_id#58, c_first_name#59, c_last_name#60, d_year#66, sum#68] +Keys [4]: [c_customer_id#58, c_first_name#59, c_last_name#60, d_year#66] +Functions [1]: [sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#62))#70] +Results [2]: [c_customer_id#58 AS customer_id#71, MakeDecimal(sum(UnscaledValue(ws_net_paid#62))#70,17,2) AS year_total#72] (69) BroadcastExchange -Input [2]: [customer_id#72, year_total#73] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#74] +Input [2]: [customer_id#71, year_total#72] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#73] (70) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#72] -Join condition: (CASE WHEN (year_total#55 > 0.00) THEN CheckOverflow((promote_precision(year_total#73) / promote_precision(year_total#55)), DecimalType(37,20), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#57) +Right keys [1]: [customer_id#71] +Join condition: (CASE WHEN (year_total#55 > 0.00) THEN CheckOverflow((promote_precision(year_total#72) / promote_precision(year_total#55)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#17 > 0.00) THEN CheckOverflow((promote_precision(year_total#36) / promote_precision(year_total#17)), DecimalType(37,20), true) ELSE null END) (71) Project [codegen id : 16] Output [3]: [customer_id#33, customer_first_name#34, customer_last_name#35] -Input [8]: [customer_id#16, customer_id#33, customer_first_name#34, customer_last_name#35, year_total#55, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#57, customer_id#72, year_total#73] +Input [9]: [customer_id#16, year_total#17, customer_id#33, customer_first_name#34, customer_last_name#35, year_total#36, year_total#55, customer_id#71, year_total#72] (72) TakeOrderedAndProject Input [3]: [customer_id#33, customer_first_name#34, customer_last_name#35] @@ -437,7 +437,7 @@ Condition : (((isnotnull(d_year#11) AND (d_year#11 = 2001)) AND d_year#11 IN (20 (76) BroadcastExchange Input [2]: [d_date_sk#10, d_year#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (80) @@ -462,10 +462,10 @@ Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (20 (80) BroadcastExchange Input [2]: [d_date_sk#27, d_year#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#25 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#25 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index 4d6a4b6355ee3..ef4d5a103e0d6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -1,8 +1,8 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (16) Project [customer_id,customer_first_name,customer_last_name] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END] - Project [customer_id,customer_id,customer_first_name,customer_last_name,year_total,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] From c16d606c159727d458c1c6d68da1849d7b70592c Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 28 Jul 2021 22:21:59 +0800 Subject: [PATCH 03/17] Fix test --- ...PushDownJoinConditionEvaluationSuite.scala | 59 +++++++++++++------ 1 file changed, 41 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluationSuite.scala index b3de7cf5a5ace..3aefad01de0e0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluationSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Coalesce, Substring, Up import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.internal.SQLConf class PushDownJoinConditionEvaluationSuite extends PlanTest { @@ -39,41 +40,63 @@ class PushDownJoinConditionEvaluationSuite extends PlanTest { private val y = testRelation1.subquery('y) test("Push down join condition evaluation(String expressions)") { - val joinType = Inner - Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => - val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) - val correctAnswer = - x.join(y.select("y.d".attr, "y.e".attr, Alias(udf, udf.sql)()), - joinType, Option("x.a".attr === s"`${udf.sql}`".attr)) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val joinType = Inner + Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => + val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) + .select("x.a".attr, "y.e".attr) + val correctAnswer = x.select("x.a".attr, "x.b".attr, "x.c".attr) + .join(y.select("y.d".attr, "y.e".attr, Alias(udf, udf.sql)()), + joinType, Option("x.a".attr === s"`${udf.sql}`".attr)).select("x.a".attr, "y.e".attr) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } } } test("Push down join condition evaluation(null expressions)") { - val joinType = Inner - val udf = Coalesce(Seq("x.b".attr, "x.c".attr)) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val joinType = Inner + val udf = Coalesce(Seq("x.b".attr, "x.c".attr)) val originalQuery = x.join(y, joinType, Option(udf === "y.e".attr)) + .select("x.a".attr, "y.e".attr) val correctAnswer = - x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join(y, + x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join( + y.select("y.d".attr, "y.e".attr), joinType, Option(s"`${udf.sql}`".attr === "y.e".attr)) + .select("x.a".attr, "y.e".attr) comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + } + + test("Negative case: Non broadcast hash join") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000000") { + val joinType = Inner + val udf = Upper("y.d".attr) + val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) + + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) + } } test("Negative case: all children are Attributes") { - val condition = Option("x.a".attr === "y.d".attr) - val originalQuery = x.join(y, Inner, condition) - val correctAnswer = x.join(y, Inner, condition) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val condition = Option("x.a".attr === "y.d".attr) + val originalQuery = x.join(y, Inner, condition) + val correctAnswer = x.join(y, Inner, condition) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } } test("Negative case: contains Literal") { - val condition = Option("x.a".attr === "string") - val originalQuery = x.join(y, Inner, condition) - val correctAnswer = x.join(y, Inner, condition) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val condition = Option("x.a".attr === "string") + val originalQuery = x.join(y, Inner, condition) + val correctAnswer = x.join(y, Inner, condition) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } } } From fc9fa2d024ae86ff7979678764077a34a3453bdc Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sat, 31 Jul 2021 23:59:13 +0800 Subject: [PATCH 04/17] PullOutJoinCondition --- .../sql/catalyst/optimizer/Optimizer.scala | 1 + .../optimizer/PullOutJoinCondition.scala | 75 +++++ .../PushDownJoinConditionEvaluation.scala | 63 ---- .../sql/catalyst/rules/RuleIdCollection.scala | 1 - .../optimizer/PullOutJoinConditionSuite.scala | 83 +++++ ...PushDownJoinConditionEvaluationSuite.scala | 102 ------ .../spark/sql/execution/SparkOptimizer.scala | 5 +- .../approved-plans-v1_4/q11.sf100/explain.txt | 88 ++--- .../q11.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q15.sf100/explain.txt | 100 +++--- .../q15.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q4.sf100/explain.txt | 262 +++++++-------- .../q4.sf100/simplified.txt | 6 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 48 +-- .../q47.sf100/simplified.txt | 12 +- .../approved-plans-v1_4/q57.sf100/explain.txt | 48 +-- .../q57.sf100/simplified.txt | 12 +- .../approved-plans-v1_4/q8.sf100/explain.txt | 162 +++++----- .../q8.sf100/simplified.txt | 12 +- .../approved-plans-v2_7/q11.sf100/explain.txt | 88 ++--- .../q11.sf100/simplified.txt | 4 +- .../approved-plans-v2_7/q24.sf100/explain.txt | 306 +++++++++++------- .../q24.sf100/simplified.txt | 22 +- .../approved-plans-v2_7/q47.sf100/explain.txt | 48 +-- .../q47.sf100/simplified.txt | 12 +- .../approved-plans-v2_7/q57.sf100/explain.txt | 48 +-- .../q57.sf100/simplified.txt | 12 +- .../approved-plans-v2_7/q74.sf100/explain.txt | 88 ++--- .../q74.sf100/simplified.txt | 4 +- 29 files changed, 897 insertions(+), 823 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluation.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluationSuite.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 bcf18b2140492..dcc6da9afd6c1 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 @@ -151,6 +151,7 @@ abstract class Optimizer(catalogManager: CatalogManager) ReplaceExpressions, RewriteNonCorrelatedExists, PullOutGroupingExpressions, + PullOutJoinCondition, ComputeCurrentTime, ReplaceCurrentLike(catalogManager)) :: ////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala new file mode 100644 index 0000000000000..7c2d6a2fbe3c8 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala @@ -0,0 +1,75 @@ +/* + * 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.{Alias, Expression, PredicateHelper} +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN + +/** + * This rule ensures that [[Join]] condition doesn't contain complex expressions in the + * optimization phase. + * + * Complex condition expressions are pulled out to a [[Project]] node under [[Join]] and are + * referenced in join condition. + * + * {{{ + * SELECT * FROM t1 JOIN t2 ON t1.a + 10 = t2.x ==> + * Project [a#0, b#1, x#2, y#3] + * +- Join Inner, ((spark_catalog.default.t1.a + 10)#8 = x#2) + * :- Project [a#0, b#1, (a#0 + 10) AS (spark_catalog.default.t1.a + 10)#8] + * : +- Filter isnotnull((a#0 + 10)) + * : +- Relation default.t1[a#0,b#1] parquet + * +- Filter isnotnull(x#2) + * +- Relation default.t2[x#2,y#3] parquet + * }}} + */ +object PullOutJoinCondition extends Rule[LogicalPlan] + with JoinSelectionHelper with PredicateHelper { + + def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(_.containsPattern(JOIN)) { + case j @ Join(left, right, _, Some(condition), _) if j.resolved => + val complexExpressions = splitConjunctivePredicates(condition).flatMap(_.children).flatMap { + case e: Expression if !e.foldable && e.children.nonEmpty => Seq(e) + case _ => Nil + } + + val leftComplexExpressions = complexExpressions.filter(canEvaluate(_, left)) + val rightComplexExpressions = complexExpressions.filter(canEvaluate(_, right)) + + val leftComplexExpressionMap = + leftComplexExpressions.map(e => e.canonicalized -> Alias(e, e.sql)()).toMap + val rightComplexExpressionMap = + rightComplexExpressions.map(e => e.canonicalized -> Alias(e, e.sql)()).toMap + val allComplexExpressionMap = leftComplexExpressionMap ++ rightComplexExpressionMap + + if (allComplexExpressionMap.nonEmpty) { + val newCondition = condition.transformDown { + case e: Expression + if e.children.nonEmpty && allComplexExpressionMap.contains(e.canonicalized) => + allComplexExpressionMap.get(e.canonicalized).map(_.toAttribute).getOrElse(e) + } + val newLeft = Project(left.output ++ leftComplexExpressionMap.values, left) + val newRight = Project(right.output ++ rightComplexExpressionMap.values, right) + j.copy(left = newLeft, right = newRight, condition = Some(newCondition)) + } else { + j + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluation.scala deleted file mode 100644 index 60f6e86eab347..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluation.scala +++ /dev/null @@ -1,63 +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.{Alias, Expression, PredicateHelper} -import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project} -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN - -/** - * Push down join condition evaluation to reduce eval expressions in join condition. - */ -object PushDownJoinConditionEvaluation extends Rule[LogicalPlan] - with JoinSelectionHelper with PredicateHelper { - - def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning( - _.containsPattern(JOIN), ruleId) { - case j @ Join(left, right, _, Some(condition), _) if !canPlanAsBroadcastHashJoin(j, conf) => - val expressions = splitConjunctivePredicates(condition).flatMap(_.children).flatMap { - case e: Expression if e.children.nonEmpty => Seq(e) - case _ => Nil - } - - val leftKeys = expressions.filter(canEvaluate(_, left)) - val rightKeys = expressions.filter(canEvaluate(_, right)) - - val leftAlias = leftKeys.map(e => Alias(e, e.sql)()) - val rightAlias = rightKeys.map(e => Alias(e, e.sql)()) - - if (leftAlias.nonEmpty || rightAlias.nonEmpty) { - val pushedPairs = leftKeys.zip(leftAlias).toMap ++ rightKeys.zip(rightAlias).toMap - val newLeft = Project(left.output ++ leftAlias, left) - val newRight = Project(right.output ++ rightAlias, right) - val newCondition = if (leftAlias.nonEmpty || rightAlias.nonEmpty) { - condition.transformDown { - case e: Expression if e.references.nonEmpty && pushedPairs.contains(e) => - pushedPairs(e).toAttribute - } - } else { - condition - } - Project(j.output, j.copy(left = newLeft, right = newRight, condition = Some(newCondition))) - } else { - j - } - - } -} 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 b379d4d195457..2a05b8533bac1 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 @@ -144,7 +144,6 @@ object RuleIdCollection { "org.apache.spark.sql.catalyst.optimizer.ReplaceIntersectWithSemiJoin" :: "org.apache.spark.sql.catalyst.optimizer.RewriteExceptAll" :: "org.apache.spark.sql.catalyst.optimizer.RewriteIntersectAll" :: - "org.apache.spark.sql.catalyst.optimizer.PushDownJoinConditionEvaluation" :: "org.apache.spark.sql.catalyst.optimizer.SimplifyBinaryComparison" :: "org.apache.spark.sql.catalyst.optimizer.SimplifyCaseConversionExpressions" :: "org.apache.spark.sql.catalyst.optimizer.SimplifyCasts" :: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala new file mode 100644 index 0000000000000..ba837b5fac228 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala @@ -0,0 +1,83 @@ +/* + * 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.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Alias, Coalesce, Substring, Upper} +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ + +class PullOutJoinConditionSuite extends PlanTest { + + private object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Pull out join condition", Once, + PullOutJoinCondition, + CollapseProject) :: Nil + } + + private val testRelation = LocalRelation('a.string, 'b.int, 'c.int) + private val testRelation1 = LocalRelation('d.string, 'e.int) + private val x = testRelation.subquery('x) + private val y = testRelation1.subquery('y) + + test("Push down join condition evaluation(String expressions)") { + val joinType = Inner + Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => + val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) + .select("x.a".attr, "y.e".attr) + val correctAnswer = x.select("x.a".attr, "x.b".attr, "x.c".attr) + .join(y.select("y.d".attr, "y.e".attr, Alias(udf, udf.sql)()), + joinType, Option("x.a".attr === s"`${udf.sql}`".attr)).select("x.a".attr, "y.e".attr) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + } + + test("Push down join condition evaluation(null expressions)") { + val joinType = Inner + val udf = Coalesce(Seq("x.b".attr, "x.c".attr)) + val originalQuery = x.join(y, joinType, Option(udf === "y.e".attr)) + .select("x.a".attr, "y.e".attr) + val correctAnswer = + x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join( + y.select("y.d".attr, "y.e".attr), + joinType, Option(s"`${udf.sql}`".attr === "y.e".attr)) + .select("x.a".attr, "y.e".attr) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Negative case: all children are Attributes") { + val condition = Option("x.a".attr === "y.d".attr) + val originalQuery = x.join(y, Inner, condition) + val correctAnswer = x.join(y, Inner, condition) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + + test("Negative case: contains Literal") { + val condition = Option("x.a".attr === "string") + val originalQuery = x.join(y, Inner, condition) + val correctAnswer = x.join(y, Inner, condition) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluationSuite.scala deleted file mode 100644 index 3aefad01de0e0..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinConditionEvaluationSuite.scala +++ /dev/null @@ -1,102 +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.dsl.expressions._ -import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{Alias, Coalesce, Substring, Upper} -import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.internal.SQLConf - -class PushDownJoinConditionEvaluationSuite extends PlanTest { - - private object Optimize extends RuleExecutor[LogicalPlan] { - val batches = - Batch("Push down join condition evaluation", Once, - PushDownJoinConditionEvaluation, - CollapseProject) :: Nil - } - - private val testRelation = LocalRelation('a.string, 'b.int, 'c.int) - private val testRelation1 = LocalRelation('d.string, 'e.int) - private val x = testRelation.subquery('x) - private val y = testRelation1.subquery('y) - - test("Push down join condition evaluation(String expressions)") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val joinType = Inner - Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => - val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) - .select("x.a".attr, "y.e".attr) - val correctAnswer = x.select("x.a".attr, "x.b".attr, "x.c".attr) - .join(y.select("y.d".attr, "y.e".attr, Alias(udf, udf.sql)()), - joinType, Option("x.a".attr === s"`${udf.sql}`".attr)).select("x.a".attr, "y.e".attr) - - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) - } - } - } - - test("Push down join condition evaluation(null expressions)") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val joinType = Inner - val udf = Coalesce(Seq("x.b".attr, "x.c".attr)) - val originalQuery = x.join(y, joinType, Option(udf === "y.e".attr)) - .select("x.a".attr, "y.e".attr) - val correctAnswer = - x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join( - y.select("y.d".attr, "y.e".attr), - joinType, Option(s"`${udf.sql}`".attr === "y.e".attr)) - .select("x.a".attr, "y.e".attr) - - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) - } - } - - test("Negative case: Non broadcast hash join") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000000") { - val joinType = Inner - val udf = Upper("y.d".attr) - val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) - - comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) - } - } - - test("Negative case: all children are Attributes") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val condition = Option("x.a".attr === "y.d".attr) - val originalQuery = x.join(y, Inner, condition) - val correctAnswer = x.join(y, Inner, condition) - - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) - } - } - - test("Negative case: contains Literal") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val condition = Option("x.a".attr === "string") - val originalQuery = x.join(y, Inner, condition) - val correctAnswer = x.join(y, Inner, condition) - - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 949ec7f75d2fd..dde5dc2be0556 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -48,10 +48,7 @@ class SparkOptimizer( PushDownPredicates) :+ Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, - PruneFilters) :+ - Batch("Push down join condition evaluation", Once, - PushDownJoinConditionEvaluation, - CollapseProject)) ++ + PruneFilters)) ++ postHocOptimizationBatches :+ Batch("Extract Python UDFs", Once, ExtractPythonUDFFromJoinCondition, diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt index fcc62c8dea526..fa8839240f91e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt @@ -355,94 +355,94 @@ Right keys [1]: [customer_id#70] Join condition: None (61) Project [codegen id : 26] -Output [4]: [customer_id#22, customer_preferred_cust_flag#46, year_total#71, CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#73] +Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#71] Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, customer_id#70, year_total#71] (62) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77] +Output [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_sold_date_sk#77 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ws_sold_date_sk#76), dynamicpruningexpression(ws_sold_date_sk#76 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (63) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77] +Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] (64) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77] -Condition : isnotnull(ws_bill_customer_sk#74) +Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] +Condition : isnotnull(ws_bill_customer_sk#73) (65) ReusedExchange [Reuses operator id: 89] -Output [2]: [d_date_sk#78, d_year#79] +Output [2]: [d_date_sk#77, d_year#78] (66) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#77] -Right keys [1]: [d_date_sk#78] +Left keys [1]: [ws_sold_date_sk#76] +Right keys [1]: [d_date_sk#77] Join condition: None (67) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77, d_date_sk#78, d_year#79] +Output [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] +Input [6]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76, d_date_sk#77, d_year#78] (68) Exchange -Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] -Arguments: hashpartitioning(ws_bill_customer_sk#74, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] +Arguments: hashpartitioning(ws_bill_customer_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] (69) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] -Arguments: [ws_bill_customer_sk#74 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] +Arguments: [ws_bill_customer_sk#73 ASC NULLS FIRST], false, 0 (70) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#81, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88] +Output [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] (71) Sort [codegen id : 31] -Input [8]: [c_customer_sk#81, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88] -Arguments: [c_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] +Arguments: [c_customer_sk#80 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#74] -Right keys [1]: [c_customer_sk#81] +Left keys [1]: [ws_bill_customer_sk#73] +Right keys [1]: [c_customer_sk#80] Join condition: None (73) Project [codegen id : 32] -Output [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] -Input [12]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79, c_customer_sk#81, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88] +Output [10]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] +Input [12]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78, c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] (74) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] -Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#89] -Results [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, sum#90] +Input [10]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] +Keys [8]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#88] +Results [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] (75) Exchange -Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, sum#90] -Arguments: hashpartitioning(c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] +Arguments: hashpartitioning(c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#90] (76) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, sum#90] -Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))#92] -Results [2]: [c_customer_id#82 AS customer_id#93, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))#92,18,2) AS year_total#94] +Input [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] +Keys [8]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))#91] +Results [2]: [c_customer_id#81 AS customer_id#92, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))#91,18,2) AS year_total#93] (77) Exchange -Input [2]: [customer_id#93, year_total#94] -Arguments: hashpartitioning(customer_id#93, 5), ENSURE_REQUIREMENTS, [id=#95] +Input [2]: [customer_id#92, year_total#93] +Arguments: hashpartitioning(customer_id#92, 5), ENSURE_REQUIREMENTS, [id=#94] (78) Sort [codegen id : 34] -Input [2]: [customer_id#93, year_total#94] -Arguments: [customer_id#93 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#92, year_total#93] +Arguments: [customer_id#92 ASC NULLS FIRST], false, 0 (79) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#93] -Join condition: (CASE WHEN (year_total#71 > 0.00) THEN CheckOverflow((promote_precision(year_total#94) / promote_precision(year_total#71)), DecimalType(38,20), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#73) +Right keys [1]: [customer_id#92] +Join condition: (CASE WHEN (year_total#71 > 0.00) THEN CheckOverflow((promote_precision(year_total#93) / promote_precision(year_total#71)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END) (80) Project [codegen id : 35] Output [1]: [customer_preferred_cust_flag#46] -Input [6]: [customer_id#22, customer_preferred_cust_flag#46, year_total#71, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#73, customer_id#93, year_total#94] +Input [7]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#71, customer_id#92, year_total#93] (81) TakeOrderedAndProject Input [1]: [customer_preferred_cust_flag#46] @@ -473,7 +473,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2001)) AND isnotnull(d_date_sk (85) BroadcastExchange Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (89) @@ -498,10 +498,10 @@ Condition : ((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND isnotnull(d_date_ (89) BroadcastExchange Input [2]: [d_date_sk#30, d_year#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#29 +Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#76 IN dynamicpruning#29 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt index 7162fb28da5ce..889fc666bd810 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (35) Project [customer_preferred_cust_flag] - SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END] + SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (26) - Project [customer_id,customer_preferred_cust_flag,year_total,year_total,year_total] + Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt index 6be8325f6af7b..55c39ed2e3288 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt @@ -53,108 +53,108 @@ Right keys [1]: [d_date_sk#5] Join condition: None (6) Project [codegen id : 2] -Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, (cs_sales_price#2 > 500.00) AS (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6] +Output [2]: [cs_bill_customer_sk#1, cs_sales_price#2] Input [4]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, d_date_sk#5] (7) Exchange -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#7] +Input [2]: [cs_bill_customer_sk#1, cs_sales_price#2] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#6] (8) Sort [codegen id : 3] -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6] +Input [2]: [cs_bill_customer_sk#1, cs_sales_price#2] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (9) Scan parquet default.customer -Output [2]: [c_customer_sk#8, c_current_addr_sk#9] +Output [2]: [c_customer_sk#7, c_current_addr_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 4] -Input [2]: [c_customer_sk#8, c_current_addr_sk#9] +Input [2]: [c_customer_sk#7, c_current_addr_sk#8] (11) Filter [codegen id : 4] -Input [2]: [c_customer_sk#8, c_current_addr_sk#9] -Condition : (isnotnull(c_customer_sk#8) AND isnotnull(c_current_addr_sk#9)) +Input [2]: [c_customer_sk#7, c_current_addr_sk#8] +Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) (12) Exchange -Input [2]: [c_customer_sk#8, c_current_addr_sk#9] -Arguments: hashpartitioning(c_current_addr_sk#9, 5), ENSURE_REQUIREMENTS, [id=#10] +Input [2]: [c_customer_sk#7, c_current_addr_sk#8] +Arguments: hashpartitioning(c_current_addr_sk#8, 5), ENSURE_REQUIREMENTS, [id=#9] (13) Sort [codegen id : 5] -Input [2]: [c_customer_sk#8, c_current_addr_sk#9] -Arguments: [c_current_addr_sk#9 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#7, c_current_addr_sk#8] +Arguments: [c_current_addr_sk#8 ASC NULLS FIRST], false, 0 (14) Scan parquet default.customer_address -Output [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] +Output [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 6] -Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] +Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] (16) Filter [codegen id : 6] -Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] -Condition : isnotnull(ca_address_sk#11) +Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] +Condition : isnotnull(ca_address_sk#10) (17) Exchange -Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] -Arguments: hashpartitioning(ca_address_sk#11, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] +Arguments: hashpartitioning(ca_address_sk#10, 5), ENSURE_REQUIREMENTS, [id=#13] (18) Sort [codegen id : 7] -Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] -Arguments: [ca_address_sk#11 ASC NULLS FIRST], false, 0 +Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] +Arguments: [ca_address_sk#10 ASC NULLS FIRST], false, 0 (19) SortMergeJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#9] -Right keys [1]: [ca_address_sk#11] +Left keys [1]: [c_current_addr_sk#8] +Right keys [1]: [ca_address_sk#10] Join condition: None (20) Project [codegen id : 8] -Output [3]: [c_customer_sk#8, ca_zip#13, (substr(ca_zip#13, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#12 IN (CA,WA,GA)) AS ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] -Input [5]: [c_customer_sk#8, c_current_addr_sk#9, ca_address_sk#11, ca_state#12, ca_zip#13] +Output [3]: [c_customer_sk#7, ca_state#11, ca_zip#12] +Input [5]: [c_customer_sk#7, c_current_addr_sk#8, ca_address_sk#10, ca_state#11, ca_zip#12] (21) Exchange -Input [3]: [c_customer_sk#8, ca_zip#13, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] -Arguments: hashpartitioning(c_customer_sk#8, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [3]: [c_customer_sk#7, ca_state#11, ca_zip#12] +Arguments: hashpartitioning(c_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] (22) Sort [codegen id : 9] -Input [3]: [c_customer_sk#8, ca_zip#13, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] -Arguments: [c_customer_sk#8 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#7, ca_state#11, ca_zip#12] +Arguments: [c_customer_sk#7 ASC NULLS FIRST], false, 0 (23) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#8] -Join condition: (((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15 OR (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6) +Right keys [1]: [c_customer_sk#7] +Join condition: ((substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#11 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) (24) Project [codegen id : 10] -Output [2]: [cs_sales_price#2, ca_zip#13] -Input [6]: [cs_bill_customer_sk#1, cs_sales_price#2, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6, c_customer_sk#8, ca_zip#13, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] +Output [2]: [cs_sales_price#2, ca_zip#12] +Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, c_customer_sk#7, ca_state#11, ca_zip#12] (25) HashAggregate [codegen id : 10] -Input [2]: [cs_sales_price#2, ca_zip#13] -Keys [1]: [ca_zip#13] +Input [2]: [cs_sales_price#2, ca_zip#12] +Keys [1]: [ca_zip#12] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum#17] -Results [2]: [ca_zip#13, sum#18] +Aggregate Attributes [1]: [sum#15] +Results [2]: [ca_zip#12, sum#16] (26) Exchange -Input [2]: [ca_zip#13, sum#18] -Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, [id=#19] +Input [2]: [ca_zip#12, sum#16] +Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [id=#17] (27) HashAggregate [codegen id : 11] -Input [2]: [ca_zip#13, sum#18] -Keys [1]: [ca_zip#13] +Input [2]: [ca_zip#12, sum#16] +Keys [1]: [ca_zip#12] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#20] -Results [2]: [ca_zip#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#20,17,2) AS sum(cs_sales_price)#21] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#18] +Results [2]: [ca_zip#12, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#18,17,2) AS sum(cs_sales_price)#19] (28) TakeOrderedAndProject -Input [2]: [ca_zip#13, sum(cs_sales_price)#21] -Arguments: 100, [ca_zip#13 ASC NULLS FIRST], [ca_zip#13, sum(cs_sales_price)#21] +Input [2]: [ca_zip#12, sum(cs_sales_price)#19] +Arguments: 100, [ca_zip#12 ASC NULLS FIRST], [ca_zip#12, sum(cs_sales_price)#19] ===== Subqueries ===== @@ -167,25 +167,25 @@ BroadcastExchange (33) (29) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#22, d_qoy#23] +Output [3]: [d_date_sk#5, d_year#20, d_qoy#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] +Input [3]: [d_date_sk#5, d_year#20, d_qoy#21] (31) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] -Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 2)) AND (d_year#22 = 2001)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#5, d_year#20, d_qoy#21] +Condition : ((((isnotnull(d_qoy#21) AND isnotnull(d_year#20)) AND (d_qoy#21 = 2)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#5)) (32) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] +Input [3]: [d_date_sk#5, d_year#20, d_qoy#21] (33) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt index fee4d9431c8f2..8105f67555c09 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (10) HashAggregate [ca_zip,cs_sales_price] [sum,sum] Project [cs_sales_price,ca_zip] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk,((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA'))),(spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk,ca_zip,ca_state,cs_sales_price] InputAdapter WholeStageCodegen (3) Sort [cs_bill_customer_sk] @@ -35,7 +35,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (8) - Project [c_customer_sk,ca_zip,ca_state] + Project [c_customer_sk,ca_state,ca_zip] SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (5) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt index 107d5fb85a415..4e4dd9555c718 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt @@ -390,274 +390,274 @@ Right keys [1]: [customer_id#87] Join condition: None (60) Project [codegen id : 26] -Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#60) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#90] +Output [11]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#88] Input [12]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, customer_id#87, year_total#88] (61) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96] +Output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#96), dynamicpruningexpression(cs_sold_date_sk#96 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(cs_sold_date_sk#95), dynamicpruningexpression(cs_sold_date_sk#95 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 28] -Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96] +Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] (63) Filter [codegen id : 28] -Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96] -Condition : isnotnull(cs_bill_customer_sk#91) +Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] +Condition : isnotnull(cs_bill_customer_sk#90) (64) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#97, d_year#98] +Output [2]: [d_date_sk#96, d_year#97] (65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#96] -Right keys [1]: [d_date_sk#97] +Left keys [1]: [cs_sold_date_sk#95] +Right keys [1]: [d_date_sk#96] Join condition: None (66) Project [codegen id : 28] -Output [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] -Input [8]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96, d_date_sk#97, d_year#98] +Output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] +Input [8]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95, d_date_sk#96, d_year#97] (67) Exchange -Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] -Arguments: hashpartitioning(cs_bill_customer_sk#91, 5), ENSURE_REQUIREMENTS, [id=#99] +Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] +Arguments: hashpartitioning(cs_bill_customer_sk#90, 5), ENSURE_REQUIREMENTS, [id=#98] (68) Sort [codegen id : 29] -Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] -Arguments: [cs_bill_customer_sk#91 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] +Arguments: [cs_bill_customer_sk#90 ASC NULLS FIRST], false, 0 (69) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#100, c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107] +Output [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] (70) Sort [codegen id : 31] -Input [8]: [c_customer_sk#100, c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107] -Arguments: [c_customer_sk#100 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] +Arguments: [c_customer_sk#99 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 32] -Left keys [1]: [cs_bill_customer_sk#91] -Right keys [1]: [c_customer_sk#100] +Left keys [1]: [cs_bill_customer_sk#90] +Right keys [1]: [c_customer_sk#99] Join condition: None (72) Project [codegen id : 32] -Output [12]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] -Input [14]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98, c_customer_sk#100, c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107] +Output [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] +Input [14]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97, c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] (73) HashAggregate [codegen id : 32] -Input [12]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] -Keys [8]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#108, isEmpty#109] -Results [10]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, sum#110, isEmpty#111] +Input [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] +Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#107, isEmpty#108] +Results [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, sum#109, isEmpty#110] (74) Exchange -Input [10]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, sum#110, isEmpty#111] -Arguments: hashpartitioning(c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, 5), ENSURE_REQUIREMENTS, [id=#112] +Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, sum#109, isEmpty#110] +Arguments: hashpartitioning(c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, 5), ENSURE_REQUIREMENTS, [id=#111] (75) HashAggregate [codegen id : 33] -Input [10]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, sum#110, isEmpty#111] -Keys [8]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#113] -Results [2]: [c_customer_id#101 AS customer_id#114, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#113 AS year_total#115] +Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, sum#109, isEmpty#110] +Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112] +Results [2]: [c_customer_id#100 AS customer_id#113, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112 AS year_total#114] (76) Exchange -Input [2]: [customer_id#114, year_total#115] -Arguments: hashpartitioning(customer_id#114, 5), ENSURE_REQUIREMENTS, [id=#116] +Input [2]: [customer_id#113, year_total#114] +Arguments: hashpartitioning(customer_id#113, 5), ENSURE_REQUIREMENTS, [id=#115] (77) Sort [codegen id : 34] -Input [2]: [customer_id#114, year_total#115] -Arguments: [customer_id#114 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#113, year_total#114] +Arguments: [customer_id#113 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#114] -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#115) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#90) +Right keys [1]: [customer_id#113] +Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#60) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END) (79) Project [codegen id : 35] -Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#115] -Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#90, customer_id#114, year_total#115] +Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114] +Input [13]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#88, customer_id#113, year_total#114] (80) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] +Output [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#122), dynamicpruningexpression(ws_sold_date_sk#122 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 37] -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] +Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121] (82) Filter [codegen id : 37] -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Condition : isnotnull(ws_bill_customer_sk#117) +Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121] +Condition : isnotnull(ws_bill_customer_sk#116) (83) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#123, d_year#124] +Output [2]: [d_date_sk#122, d_year#123] (84) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_sold_date_sk#122] -Right keys [1]: [d_date_sk#123] +Left keys [1]: [ws_sold_date_sk#121] +Right keys [1]: [d_date_sk#122] Join condition: None (85) Project [codegen id : 37] -Output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] -Input [8]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122, d_date_sk#123, d_year#124] +Output [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] +Input [8]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121, d_date_sk#122, d_year#123] (86) Exchange -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] -Arguments: hashpartitioning(ws_bill_customer_sk#117, 5), ENSURE_REQUIREMENTS, [id=#125] +Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] +Arguments: hashpartitioning(ws_bill_customer_sk#116, 5), ENSURE_REQUIREMENTS, [id=#124] (87) Sort [codegen id : 38] -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] -Arguments: [ws_bill_customer_sk#117 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] +Arguments: [ws_bill_customer_sk#116 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#126, c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133] +Output [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] (89) Sort [codegen id : 40] -Input [8]: [c_customer_sk#126, c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133] -Arguments: [c_customer_sk#126 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +Arguments: [c_customer_sk#125 ASC NULLS FIRST], false, 0 (90) SortMergeJoin [codegen id : 41] -Left keys [1]: [ws_bill_customer_sk#117] -Right keys [1]: [c_customer_sk#126] +Left keys [1]: [ws_bill_customer_sk#116] +Right keys [1]: [c_customer_sk#125] Join condition: None (91) Project [codegen id : 41] -Output [12]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] -Input [14]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124, c_customer_sk#126, c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133] +Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] +Input [14]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123, c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] (92) HashAggregate [codegen id : 41] -Input [12]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] -Keys [8]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#134, isEmpty#135] -Results [10]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, sum#136, isEmpty#137] +Input [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] +Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#133, isEmpty#134] +Results [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, sum#135, isEmpty#136] (93) Exchange -Input [10]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, sum#136, isEmpty#137] -Arguments: hashpartitioning(c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, 5), ENSURE_REQUIREMENTS, [id=#138] +Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, sum#135, isEmpty#136] +Arguments: hashpartitioning(c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, 5), ENSURE_REQUIREMENTS, [id=#137] (94) HashAggregate [codegen id : 42] -Input [10]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, sum#136, isEmpty#137] -Keys [8]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#139] -Results [2]: [c_customer_id#127 AS customer_id#140, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#139 AS year_total#141] +Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, sum#135, isEmpty#136] +Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138] +Results [2]: [c_customer_id#126 AS customer_id#139, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138 AS year_total#140] (95) Filter [codegen id : 42] -Input [2]: [customer_id#140, year_total#141] -Condition : (isnotnull(year_total#141) AND (year_total#141 > 0.000000)) +Input [2]: [customer_id#139, year_total#140] +Condition : (isnotnull(year_total#140) AND (year_total#140 > 0.000000)) (96) Project [codegen id : 42] -Output [2]: [customer_id#140 AS customer_id#142, year_total#141 AS year_total#143] -Input [2]: [customer_id#140, year_total#141] +Output [2]: [customer_id#139 AS customer_id#141, year_total#140 AS year_total#142] +Input [2]: [customer_id#139, year_total#140] (97) Exchange -Input [2]: [customer_id#142, year_total#143] -Arguments: hashpartitioning(customer_id#142, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [2]: [customer_id#141, year_total#142] +Arguments: hashpartitioning(customer_id#141, 5), ENSURE_REQUIREMENTS, [id=#143] (98) Sort [codegen id : 43] -Input [2]: [customer_id#142, year_total#143] -Arguments: [customer_id#142 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#141, year_total#142] +Arguments: [customer_id#141 ASC NULLS FIRST], false, 0 (99) SortMergeJoin [codegen id : 44] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#142] +Right keys [1]: [customer_id#141] Join condition: None (100) Project [codegen id : 44] -Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#143, CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#115) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END AS CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#145] -Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#115, customer_id#142, year_total#143] +Output [11]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114, year_total#142] +Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114, customer_id#141, year_total#142] (101) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] +Output [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#151), dynamicpruningexpression(ws_sold_date_sk#151 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(ws_sold_date_sk#149), dynamicpruningexpression(ws_sold_date_sk#149 IN dynamicpruning#35)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 46] -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] +Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] (103) Filter [codegen id : 46] -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] -Condition : isnotnull(ws_bill_customer_sk#146) +Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] +Condition : isnotnull(ws_bill_customer_sk#144) (104) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#150, d_year#151] (105) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [ws_sold_date_sk#151] -Right keys [1]: [d_date_sk#152] +Left keys [1]: [ws_sold_date_sk#149] +Right keys [1]: [d_date_sk#150] Join condition: None (106) Project [codegen id : 46] -Output [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Input [8]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151, d_date_sk#152, d_year#153] +Output [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] +Input [8]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149, d_date_sk#150, d_year#151] (107) Exchange -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Arguments: hashpartitioning(ws_bill_customer_sk#146, 5), ENSURE_REQUIREMENTS, [id=#154] +Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] +Arguments: hashpartitioning(ws_bill_customer_sk#144, 5), ENSURE_REQUIREMENTS, [id=#152] (108) Sort [codegen id : 47] -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Arguments: [ws_bill_customer_sk#146 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] +Arguments: [ws_bill_customer_sk#144 ASC NULLS FIRST], false, 0 (109) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] +Output [8]: [c_customer_sk#153, c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160] (110) Sort [codegen id : 49] -Input [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] -Arguments: [c_customer_sk#155 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#153, c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160] +Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 (111) SortMergeJoin [codegen id : 50] -Left keys [1]: [ws_bill_customer_sk#146] -Right keys [1]: [c_customer_sk#155] +Left keys [1]: [ws_bill_customer_sk#144] +Right keys [1]: [c_customer_sk#153] Join condition: None (112) Project [codegen id : 50] -Output [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Input [14]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153, c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] +Output [12]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] +Input [14]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151, c_customer_sk#153, c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160] (113) HashAggregate [codegen id : 50] -Input [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#163, isEmpty#164] -Results [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Input [12]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] +Keys [8]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#161, isEmpty#162] +Results [10]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, sum#163, isEmpty#164] (114) Exchange -Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] -Arguments: hashpartitioning(c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [10]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, sum#163, isEmpty#164] +Arguments: hashpartitioning(c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, 5), ENSURE_REQUIREMENTS, [id=#165] (115) HashAggregate [codegen id : 51] -Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] -Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#168] -Results [2]: [c_customer_id#156 AS customer_id#169, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#168 AS year_total#170] +Input [10]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, sum#163, isEmpty#164] +Keys [8]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166] +Results [2]: [c_customer_id#154 AS customer_id#167, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166 AS year_total#168] (116) Exchange -Input [2]: [customer_id#169, year_total#170] -Arguments: hashpartitioning(customer_id#169, 5), ENSURE_REQUIREMENTS, [id=#171] +Input [2]: [customer_id#167, year_total#168] +Arguments: hashpartitioning(customer_id#167, 5), ENSURE_REQUIREMENTS, [id=#169] (117) Sort [codegen id : 52] -Input [2]: [customer_id#169, year_total#170] -Arguments: [customer_id#169 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#167, year_total#168] +Arguments: [customer_id#167 ASC NULLS FIRST], false, 0 (118) SortMergeJoin [codegen id : 53] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#169] -Join condition: (CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#145 > CASE WHEN (year_total#143 > 0.000000) THEN CheckOverflow((promote_precision(year_total#170) / promote_precision(year_total#143)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#167] +Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#142 > 0.000000) THEN CheckOverflow((promote_precision(year_total#168) / promote_precision(year_total#142)), DecimalType(38,14), true) ELSE null END) (119) Project [codegen id : 53] Output [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] -Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#143, CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#145, customer_id#169, year_total#170] +Input [13]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114, year_total#142, customer_id#167, year_total#168] (120) TakeOrderedAndProject Input [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] @@ -688,7 +688,7 @@ Condition : ((isnotnull(d_year#9) AND (d_year#9 = 2001)) AND isnotnull(d_date_sk (124) BroadcastExchange Input [2]: [d_date_sk#8, d_year#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#172] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#170] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#34 IN dynamicpruning#35 BroadcastExchange (128) @@ -713,14 +713,14 @@ Condition : ((isnotnull(d_year#37) AND (d_year#37 = 2002)) AND isnotnull(d_date_ (128) BroadcastExchange Input [2]: [d_date_sk#36, d_year#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#173] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#171] Subquery:3 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#7 -Subquery:4 Hosting operator id = 61 Hosting Expression = cs_sold_date_sk#96 IN dynamicpruning#35 +Subquery:4 Hosting operator id = 61 Hosting Expression = cs_sold_date_sk#95 IN dynamicpruning#35 -Subquery:5 Hosting operator id = 80 Hosting Expression = ws_sold_date_sk#122 IN dynamicpruning#7 +Subquery:5 Hosting operator id = 80 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#7 -Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#151 IN dynamicpruning#35 +Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#149 IN dynamicpruning#35 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt index eab288e1d9891..d7e0a660bab5b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] WholeStageCodegen (53) Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - SortMergeJoin [customer_id,customer_id,CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END,year_total,year_total] + SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (44) Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] @@ -9,10 +9,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter WholeStageCodegen (35) Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END] + SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (26) - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) 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 a612378242f1e..686036a835e25 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 @@ -223,61 +223,61 @@ Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_yea Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (rn#39 + 1) AS (v1_lag.rn + 1)#40] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] (40) Exchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] (41) Sort [codegen id : 23] -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (v1_lag.rn + 1)#40 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None (43) Project [codegen id : 24] Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37] -Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] +Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] (44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Output [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] (45) Sort [codegen id : 33] -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST], false, 0 (46) Window -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] +Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Arguments: [rank(d_year#45, d_moy#46) windowspecdefinition(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#48], [i_category#41, i_brand#42, s_store_name#43, s_company_name#44], [d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (rn#49 - 1) AS (v1_lead.rn - 1)#50] -Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] +Output [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Input [8]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47, rn#48] (48) Exchange -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] -Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Arguments: hashpartitioning(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1), 5), ENSURE_REQUIREMENTS, [id=#49] (49) Sort [codegen id : 35] -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (v1_lead.rn - 1)#50 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, (rn#48 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50] +Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None (51) Project [codegen id : 36] -Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#52, sum_sales#48 AS nsum#53] -Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] +Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] +Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] (52) TakeOrderedAndProject -Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] +Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt index 5c9d3bb5049c0..07c75d91ca3cf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] WholeStageCodegen (36) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,s_store_name,s_company_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] #8 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #8 WholeStageCodegen (22) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] #10 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #10 WholeStageCodegen (34) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter 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 b7ba9e2c06424..0f22f27fed99b 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 @@ -223,61 +223,61 @@ Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sale Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (rn#37 + 1) AS (v1_lag.rn + 1)#38] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] (40) Exchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#38] (41) Sort [codegen id : 23] -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (v1_lag.rn + 1)#38 ASC NULLS FIRST], false, 0 +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None (43) Project [codegen id : 24] Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] (44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] (45) Sort [codegen id : 33] -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 (46) Window -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] +Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#39, i_brand#40, cc_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#45], [i_category#39, i_brand#40, cc_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (rn#46 - 1) AS (v1_lead.rn - 1)#47] -Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] +Output [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] (48) Exchange -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] -Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: hashpartitioning(i_category#39, i_brand#40, cc_name#41, (rn#45 - 1), 5), ENSURE_REQUIREMENTS, [id=#46] (49) Sort [codegen id : 35] -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (v1_lead.rn - 1)#47 ASC NULLS FIRST], false, 0 +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, (rn#45 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47] +Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None (51) Project [codegen id : 36] -Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#49, sum_sales#45 AS nsum#50] -Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] +Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] +Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] (52) TakeOrderedAndProject -Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] +Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt index f3ef99b986f71..3bf10f82e6a88 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] WholeStageCodegen (36) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lead.rn - 1)] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lag.rn + 1)] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,cc_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,cc_name,(v1_lag.rn + 1)] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [i_category,i_brand,cc_name,(v1_lag.rn + 1)] #8 + Exchange [i_category,i_brand,cc_name,rn] #8 WholeStageCodegen (22) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,cc_name,(v1_lead.rn - 1)] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [i_category,i_brand,cc_name,(v1_lead.rn - 1)] #10 + Exchange [i_category,i_brand,cc_name,rn] #10 WholeStageCodegen (34) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt index b6b08c52ef055..e8c77e7de05eb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt @@ -101,182 +101,182 @@ Right keys [1]: [s_store_sk#6] Join condition: None (12) Project [codegen id : 3] -Output [3]: [ss_net_profit#2, s_store_name#7, substr(s_zip#8, 1, 2) AS substr(spark_catalog.default.store.s_zip, 1, 2)#10] +Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#8] (13) Exchange -Input [3]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10] -Arguments: hashpartitioning(substr(spark_catalog.default.store.s_zip, 1, 2)#10, 5), ENSURE_REQUIREMENTS, [id=#11] +Input [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] +Arguments: hashpartitioning(substr(s_zip#8, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#10] (14) Sort [codegen id : 4] -Input [3]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10] -Arguments: [substr(spark_catalog.default.store.s_zip, 1, 2)#10 ASC NULLS FIRST], false, 0 +Input [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] +Arguments: [substr(s_zip#8, 1, 2) ASC NULLS FIRST], false, 0 (15) Scan parquet default.customer_address -Output [1]: [ca_zip#12] +Output [1]: [ca_zip#11] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] ReadSchema: struct (16) ColumnarToRow [codegen id : 11] -Input [1]: [ca_zip#12] +Input [1]: [ca_zip#11] (17) Filter [codegen id : 11] -Input [1]: [ca_zip#12] -Condition : (substr(ca_zip#12, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#12, 1, 5))) +Input [1]: [ca_zip#11] +Condition : (substr(ca_zip#11, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#11, 1, 5))) (18) Scan parquet default.customer_address -Output [2]: [ca_address_sk#13, ca_zip#14] +Output [2]: [ca_address_sk#12, ca_zip#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#13, ca_zip#14] +Input [2]: [ca_address_sk#12, ca_zip#13] (20) Filter [codegen id : 5] -Input [2]: [ca_address_sk#13, ca_zip#14] -Condition : isnotnull(ca_address_sk#13) +Input [2]: [ca_address_sk#12, ca_zip#13] +Condition : isnotnull(ca_address_sk#12) (21) Exchange -Input [2]: [ca_address_sk#13, ca_zip#14] -Arguments: hashpartitioning(ca_address_sk#13, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [2]: [ca_address_sk#12, ca_zip#13] +Arguments: hashpartitioning(ca_address_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] (22) Sort [codegen id : 6] -Input [2]: [ca_address_sk#13, ca_zip#14] -Arguments: [ca_address_sk#13 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#12, ca_zip#13] +Arguments: [ca_address_sk#12 ASC NULLS FIRST], false, 0 (23) Scan parquet default.customer -Output [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 7] -Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] (25) Filter [codegen id : 7] -Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] -Condition : ((isnotnull(c_preferred_cust_flag#17) AND (c_preferred_cust_flag#17 = Y)) AND isnotnull(c_current_addr_sk#16)) +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Condition : ((isnotnull(c_preferred_cust_flag#16) AND (c_preferred_cust_flag#16 = Y)) AND isnotnull(c_current_addr_sk#15)) (26) Project [codegen id : 7] -Output [1]: [c_current_addr_sk#16] -Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Output [1]: [c_current_addr_sk#15] +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] (27) Exchange -Input [1]: [c_current_addr_sk#16] -Arguments: hashpartitioning(c_current_addr_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [1]: [c_current_addr_sk#15] +Arguments: hashpartitioning(c_current_addr_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] (28) Sort [codegen id : 8] -Input [1]: [c_current_addr_sk#16] -Arguments: [c_current_addr_sk#16 ASC NULLS FIRST], false, 0 +Input [1]: [c_current_addr_sk#15] +Arguments: [c_current_addr_sk#15 ASC NULLS FIRST], false, 0 (29) SortMergeJoin [codegen id : 9] -Left keys [1]: [ca_address_sk#13] -Right keys [1]: [c_current_addr_sk#16] +Left keys [1]: [ca_address_sk#12] +Right keys [1]: [c_current_addr_sk#15] Join condition: None (30) Project [codegen id : 9] -Output [1]: [ca_zip#14] -Input [3]: [ca_address_sk#13, ca_zip#14, c_current_addr_sk#16] +Output [1]: [ca_zip#13] +Input [3]: [ca_address_sk#12, ca_zip#13, c_current_addr_sk#15] (31) HashAggregate [codegen id : 9] -Input [1]: [ca_zip#14] -Keys [1]: [ca_zip#14] +Input [1]: [ca_zip#13] +Keys [1]: [ca_zip#13] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#19] -Results [2]: [ca_zip#14, count#20] +Aggregate Attributes [1]: [count#18] +Results [2]: [ca_zip#13, count#19] (32) Exchange -Input [2]: [ca_zip#14, count#20] -Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [2]: [ca_zip#13, count#19] +Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, [id=#20] (33) HashAggregate [codegen id : 10] -Input [2]: [ca_zip#14, count#20] -Keys [1]: [ca_zip#14] +Input [2]: [ca_zip#13, count#19] +Keys [1]: [ca_zip#13] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#22] -Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#23, count(1)#22 AS cnt#24] +Aggregate Attributes [1]: [count(1)#21] +Results [2]: [substr(ca_zip#13, 1, 5) AS ca_zip#22, count(1)#21 AS cnt#23] (34) Filter [codegen id : 10] -Input [2]: [ca_zip#23, cnt#24] -Condition : (cnt#24 > 10) +Input [2]: [ca_zip#22, cnt#23] +Condition : (cnt#23 > 10) (35) Project [codegen id : 10] -Output [1]: [ca_zip#23] -Input [2]: [ca_zip#23, cnt#24] +Output [1]: [ca_zip#22] +Input [2]: [ca_zip#22, cnt#23] (36) BroadcastExchange -Input [1]: [ca_zip#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#25] +Input [1]: [ca_zip#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#24] (37) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [coalesce(substr(ca_zip#12, 1, 5), ), isnull(substr(ca_zip#12, 1, 5))] -Right keys [2]: [coalesce(ca_zip#23, ), isnull(ca_zip#23)] +Left keys [2]: [coalesce(substr(ca_zip#11, 1, 5), ), isnull(substr(ca_zip#11, 1, 5))] +Right keys [2]: [coalesce(ca_zip#22, ), isnull(ca_zip#22)] Join condition: None (38) Project [codegen id : 11] -Output [1]: [substr(ca_zip#12, 1, 5) AS ca_zip#26] -Input [1]: [ca_zip#12] +Output [1]: [substr(ca_zip#11, 1, 5) AS ca_zip#25] +Input [1]: [ca_zip#11] (39) HashAggregate [codegen id : 11] -Input [1]: [ca_zip#26] -Keys [1]: [ca_zip#26] +Input [1]: [ca_zip#25] +Keys [1]: [ca_zip#25] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#26] +Results [1]: [ca_zip#25] (40) Exchange -Input [1]: [ca_zip#26] -Arguments: hashpartitioning(ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [1]: [ca_zip#25] +Arguments: hashpartitioning(ca_zip#25, 5), ENSURE_REQUIREMENTS, [id=#26] (41) HashAggregate [codegen id : 12] -Input [1]: [ca_zip#26] -Keys [1]: [ca_zip#26] +Input [1]: [ca_zip#25] +Keys [1]: [ca_zip#25] Functions: [] Aggregate Attributes: [] -Results [1]: [substr(ca_zip#26, 1, 2) AS substr(V1.ca_zip, 1, 2)#28] +Results [1]: [ca_zip#25] (42) Exchange -Input [1]: [substr(V1.ca_zip, 1, 2)#28] -Arguments: hashpartitioning(substr(V1.ca_zip, 1, 2)#28, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [1]: [ca_zip#25] +Arguments: hashpartitioning(substr(ca_zip#25, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#27] (43) Sort [codegen id : 13] -Input [1]: [substr(V1.ca_zip, 1, 2)#28] -Arguments: [substr(V1.ca_zip, 1, 2)#28 ASC NULLS FIRST], false, 0 +Input [1]: [ca_zip#25] +Arguments: [substr(ca_zip#25, 1, 2) ASC NULLS FIRST], false, 0 (44) SortMergeJoin [codegen id : 14] -Left keys [1]: [substr(spark_catalog.default.store.s_zip, 1, 2)#10] -Right keys [1]: [substr(V1.ca_zip, 1, 2)#28] +Left keys [1]: [substr(s_zip#8, 1, 2)] +Right keys [1]: [substr(ca_zip#25, 1, 2)] Join condition: None (45) Project [codegen id : 14] Output [2]: [ss_net_profit#2, s_store_name#7] -Input [4]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10, substr(V1.ca_zip, 1, 2)#28] +Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#8, ca_zip#25] (46) HashAggregate [codegen id : 14] Input [2]: [ss_net_profit#2, s_store_name#7] Keys [1]: [s_store_name#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#30] -Results [2]: [s_store_name#7, sum#31] +Aggregate Attributes [1]: [sum#28] +Results [2]: [s_store_name#7, sum#29] (47) Exchange -Input [2]: [s_store_name#7, sum#31] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [2]: [s_store_name#7, sum#29] +Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#30] (48) HashAggregate [codegen id : 15] -Input [2]: [s_store_name#7, sum#31] +Input [2]: [s_store_name#7, sum#29] Keys [1]: [s_store_name#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#33] -Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#33,17,2) AS sum(ss_net_profit)#34] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] +Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS sum(ss_net_profit)#32] (49) TakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#34] -Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#34] +Input [2]: [s_store_name#7, sum(ss_net_profit)#32] +Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#32] ===== Subqueries ===== @@ -289,25 +289,25 @@ BroadcastExchange (54) (50) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#35, d_qoy#36] +Output [3]: [d_date_sk#5, d_year#33, d_qoy#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#35, d_qoy#36] +Input [3]: [d_date_sk#5, d_year#33, d_qoy#34] (52) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#35, d_qoy#36] -Condition : ((((isnotnull(d_qoy#36) AND isnotnull(d_year#35)) AND (d_qoy#36 = 2)) AND (d_year#35 = 1998)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#5, d_year#33, d_qoy#34] +Condition : ((((isnotnull(d_qoy#34) AND isnotnull(d_year#33)) AND (d_qoy#34 = 2)) AND (d_year#33 = 1998)) AND isnotnull(d_date_sk#5)) (53) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#35, d_qoy#36] +Input [3]: [d_date_sk#5, d_year#33, d_qoy#34] (54) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt index 435254f45f10d..84ac2edd606cf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt @@ -6,12 +6,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (14) HashAggregate [s_store_name,ss_net_profit] [sum,sum] Project [ss_net_profit,s_store_name] - SortMergeJoin [substr(spark_catalog.default.store.s_zip, 1, 2),substr(V1.ca_zip, 1, 2)] + SortMergeJoin [s_zip,ca_zip] InputAdapter WholeStageCodegen (4) - Sort [substr(spark_catalog.default.store.s_zip, 1, 2)] + Sort [s_zip] InputAdapter - Exchange [substr(spark_catalog.default.store.s_zip, 1, 2)] #2 + Exchange [s_zip] #2 WholeStageCodegen (3) Project [ss_net_profit,s_store_name,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -40,11 +40,11 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] Scan parquet default.store [s_store_sk,s_store_name,s_zip] InputAdapter WholeStageCodegen (13) - Sort [substr(V1.ca_zip, 1, 2)] + Sort [ca_zip] InputAdapter - Exchange [substr(V1.ca_zip, 1, 2)] #5 + Exchange [ca_zip] #5 WholeStageCodegen (12) - HashAggregate [ca_zip] [substr(V1.ca_zip, 1, 2)] + HashAggregate [ca_zip] InputAdapter Exchange [ca_zip] #6 WholeStageCodegen (11) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt index 84342b5469473..8e15a8b4c1e2a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt @@ -350,94 +350,94 @@ Right keys [1]: [customer_id#72] Join condition: None (60) Project [codegen id : 26] -Output [7]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#73, CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#75] +Output [8]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, year_total#73] Input [9]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, customer_id#72, year_total#73] (61) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_sold_date_sk#78 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] (63) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_customer_sk#76) +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] +Condition : isnotnull(ws_bill_customer_sk#75) (64) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#80, d_year#81] +Output [2]: [d_date_sk#79, d_year#80] (65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#79] -Right keys [1]: [d_date_sk#80] +Left keys [1]: [ws_sold_date_sk#78] +Right keys [1]: [d_date_sk#79] Join condition: None (66) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#80, d_year#81] +Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Input [6]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78, d_date_sk#79, d_year#80] (67) Exchange -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] -Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Arguments: hashpartitioning(ws_bill_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] (68) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] -Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Arguments: [ws_bill_customer_sk#75 ASC NULLS FIRST], false, 0 (69) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#83, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90] +Output [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] (70) Sort [codegen id : 31] -Input [8]: [c_customer_sk#83, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90] -Arguments: [c_customer_sk#83 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] +Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#76] -Right keys [1]: [c_customer_sk#83] +Left keys [1]: [ws_bill_customer_sk#75] +Right keys [1]: [c_customer_sk#82] Join condition: None (72) Project [codegen id : 32] -Output [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] -Input [12]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81, c_customer_sk#83, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90] +Output [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Input [12]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80, c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] (73) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] -Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#91] -Results [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, sum#92] +Input [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#90] +Results [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] (74) Exchange -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, sum#92] -Arguments: hashpartitioning(c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Arguments: hashpartitioning(c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, 5), ENSURE_REQUIREMENTS, [id=#92] (75) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, sum#92] -Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))#94] -Results [2]: [c_customer_id#84 AS customer_id#95, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))#94,18,2) AS year_total#96] +Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93] +Results [2]: [c_customer_id#83 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] (76) Exchange -Input [2]: [customer_id#95, year_total#96] -Arguments: hashpartitioning(customer_id#95, 5), ENSURE_REQUIREMENTS, [id=#97] +Input [2]: [customer_id#94, year_total#95] +Arguments: hashpartitioning(customer_id#94, 5), ENSURE_REQUIREMENTS, [id=#96] (77) Sort [codegen id : 34] -Input [2]: [customer_id#95, year_total#96] -Arguments: [customer_id#95 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#94, year_total#95] +Arguments: [customer_id#94 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#95] -Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#96) / promote_precision(year_total#73)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#75) +Right keys [1]: [customer_id#94] +Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END) (79) Project [codegen id : 35] Output [4]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48] -Input [9]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#73, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#75, customer_id#95, year_total#96] +Input [10]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, year_total#73, customer_id#94, year_total#95] (80) TakeOrderedAndProject Input [4]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48] @@ -468,7 +468,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2001)) AND isnotnull(d_date_sk (84) BroadcastExchange Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#97] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (88) @@ -493,10 +493,10 @@ Condition : ((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND isnotnull(d_date_ (88) BroadcastExchange Input [2]: [d_date_sk#30, d_year#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#98] Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#29 +Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#78 IN dynamicpruning#29 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt index 35e8fb614399f..b3f8a57ba0f5b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] WholeStageCodegen (35) Project [customer_id,customer_first_name,customer_last_name,customer_email_address] - SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END] + SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (26) - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt index c8cd870c62c8a..717e343973617 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt @@ -273,163 +273,229 @@ Arguments: [c_last_name#15 ASC NULLS FIRST, c_first_name#14 ASC NULLS FIRST, s_s ===== Subqueries ===== Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#48, [id=#49] -* HashAggregate (76) -+- Exchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- Exchange (72) - +- * HashAggregate (71) - +- * Project (70) - +- * SortMergeJoin Inner (69) - :- * Sort (66) - : +- Exchange (65) - : +- * Project (64) - : +- * SortMergeJoin Inner (63) - : :- * Sort (57) - : : +- Exchange (56) - : : +- * Project (55) - : : +- * BroadcastHashJoin Inner BuildLeft (54) - : : :- ReusedExchange (49) - : : +- * Project (53) - : : +- * Filter (52) - : : +- * ColumnarToRow (51) - : : +- Scan parquet default.store_sales (50) - : +- * Sort (62) - : +- Exchange (61) - : +- * Filter (60) - : +- * ColumnarToRow (59) - : +- Scan parquet default.item (58) - +- * Sort (68) - +- ReusedExchange (67) - - -(49) ReusedExchange [Reuses operator id: 17] -Output [7]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#54, c_customer_sk#55, c_first_name#56, c_last_name#57] - -(50) Scan parquet default.store_sales -Output [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] +* HashAggregate (88) ++- Exchange (87) + +- * HashAggregate (86) + +- * HashAggregate (85) + +- Exchange (84) + +- * HashAggregate (83) + +- * Project (82) + +- * SortMergeJoin Inner (81) + :- * Sort (78) + : +- Exchange (77) + : +- * Project (76) + : +- * SortMergeJoin Inner (75) + : :- * Sort (69) + : : +- Exchange (68) + : : +- * Project (67) + : : +- * BroadcastHashJoin Inner BuildLeft (66) + : : :- BroadcastExchange (61) + : : : +- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildLeft (59) + : : : :- BroadcastExchange (55) + : : : : +- * Project (54) + : : : : +- * BroadcastHashJoin Inner BuildLeft (53) + : : : : :- ReusedExchange (49) + : : : : +- * Filter (52) + : : : : +- * ColumnarToRow (51) + : : : : +- Scan parquet default.customer_address (50) + : : : +- * Filter (58) + : : : +- * ColumnarToRow (57) + : : : +- Scan parquet default.customer (56) + : : +- * Project (65) + : : +- * Filter (64) + : : +- * ColumnarToRow (63) + : : +- Scan parquet default.store_sales (62) + : +- * Sort (74) + : +- Exchange (73) + : +- * Filter (72) + : +- * ColumnarToRow (71) + : +- Scan parquet default.item (70) + +- * Sort (80) + +- ReusedExchange (79) + + +(49) ReusedExchange [Reuses operator id: 5] +Output [4]: [s_store_sk#51, s_store_name#52, s_state#53, s_zip#54] + +(50) Scan parquet default.customer_address +Output [4]: [ca_address_sk#55, ca_state#56, ca_zip#57, ca_country#58] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] +ReadSchema: struct + +(51) ColumnarToRow +Input [4]: [ca_address_sk#55, ca_state#56, ca_zip#57, ca_country#58] + +(52) Filter +Input [4]: [ca_address_sk#55, ca_state#56, ca_zip#57, ca_country#58] +Condition : (((isnotnull(ca_address_sk#55) AND isnotnull(ca_country#58)) AND isnotnull(ca_zip#57)) AND isnotnull(upper(ca_country#58))) + +(53) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [s_zip#54] +Right keys [1]: [ca_zip#57] +Join condition: None + +(54) Project [codegen id : 2] +Output [6]: [s_store_sk#51, s_store_name#52, s_state#53, ca_address_sk#55, ca_state#56, upper(ca_country#58) AS upper(spark_catalog.default.customer_address.ca_country)#59] +Input [8]: [s_store_sk#51, s_store_name#52, s_state#53, s_zip#54, ca_address_sk#55, ca_state#56, ca_zip#57, ca_country#58] + +(55) BroadcastExchange +Input [6]: [s_store_sk#51, s_store_name#52, s_state#53, ca_address_sk#55, ca_state#56, upper(spark_catalog.default.customer_address.ca_country)#59] +Arguments: HashedRelationBroadcastMode(List(input[3, int, true], input[5, string, true]),false), [id=#60] + +(56) Scan parquet default.customer +Output [5]: [c_customer_sk#61, c_current_addr_sk#62, c_first_name#63, c_last_name#64, c_birth_country#65] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] +ReadSchema: struct + +(57) ColumnarToRow +Input [5]: [c_customer_sk#61, c_current_addr_sk#62, c_first_name#63, c_last_name#64, c_birth_country#65] + +(58) Filter +Input [5]: [c_customer_sk#61, c_current_addr_sk#62, c_first_name#63, c_last_name#64, c_birth_country#65] +Condition : ((isnotnull(c_customer_sk#61) AND isnotnull(c_current_addr_sk#62)) AND isnotnull(c_birth_country#65)) + +(59) BroadcastHashJoin [codegen id : 3] +Left keys [2]: [ca_address_sk#55, upper(spark_catalog.default.customer_address.ca_country)#59] +Right keys [2]: [c_current_addr_sk#62, c_birth_country#65] +Join condition: None + +(60) Project [codegen id : 3] +Output [7]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#56, c_customer_sk#61, c_first_name#63, c_last_name#64] +Input [11]: [s_store_sk#51, s_store_name#52, s_state#53, ca_address_sk#55, ca_state#56, upper(spark_catalog.default.customer_address.ca_country)#59, c_customer_sk#61, c_current_addr_sk#62, c_first_name#63, c_last_name#64, c_birth_country#65] + +(61) BroadcastExchange +Input [7]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#56, c_customer_sk#61, c_first_name#63, c_last_name#64] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#66] + +(62) Scan parquet default.store_sales +Output [6]: [ss_item_sk#67, ss_customer_sk#68, ss_store_sk#69, ss_ticket_number#70, ss_net_paid#71, ss_sold_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(51) ColumnarToRow -Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] +(63) ColumnarToRow +Input [6]: [ss_item_sk#67, ss_customer_sk#68, ss_store_sk#69, ss_ticket_number#70, ss_net_paid#71, ss_sold_date_sk#72] -(52) Filter -Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] -Condition : (((isnotnull(ss_ticket_number#61) AND isnotnull(ss_item_sk#58)) AND isnotnull(ss_store_sk#60)) AND isnotnull(ss_customer_sk#59)) +(64) Filter +Input [6]: [ss_item_sk#67, ss_customer_sk#68, ss_store_sk#69, ss_ticket_number#70, ss_net_paid#71, ss_sold_date_sk#72] +Condition : (((isnotnull(ss_ticket_number#70) AND isnotnull(ss_item_sk#67)) AND isnotnull(ss_store_sk#69)) AND isnotnull(ss_customer_sk#68)) -(53) Project -Output [5]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62] -Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] +(65) Project +Output [5]: [ss_item_sk#67, ss_customer_sk#68, ss_store_sk#69, ss_ticket_number#70, ss_net_paid#71] +Input [6]: [ss_item_sk#67, ss_customer_sk#68, ss_store_sk#69, ss_ticket_number#70, ss_net_paid#71, ss_sold_date_sk#72] -(54) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [s_store_sk#51, c_customer_sk#55] -Right keys [2]: [ss_store_sk#60, ss_customer_sk#59] +(66) BroadcastHashJoin [codegen id : 4] +Left keys [2]: [s_store_sk#51, c_customer_sk#61] +Right keys [2]: [ss_store_sk#69, ss_customer_sk#68] Join condition: None -(55) Project [codegen id : 4] -Output [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] -Input [12]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#54, c_customer_sk#55, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62] +(67) Project [codegen id : 4] +Output [8]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71] +Input [12]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#56, c_customer_sk#61, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_customer_sk#68, ss_store_sk#69, ss_ticket_number#70, ss_net_paid#71] -(56) Exchange -Input [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] -Arguments: hashpartitioning(ss_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#64] +(68) Exchange +Input [8]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71] +Arguments: hashpartitioning(ss_item_sk#67, 5), ENSURE_REQUIREMENTS, [id=#73] -(57) Sort [codegen id : 5] -Input [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] -Arguments: [ss_item_sk#58 ASC NULLS FIRST], false, 0 +(69) Sort [codegen id : 5] +Input [8]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71] +Arguments: [ss_item_sk#67 ASC NULLS FIRST], false, 0 -(58) Scan parquet default.item -Output [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +(70) Scan parquet default.item +Output [6]: [i_item_sk#74, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +(71) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#74, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] -(60) Filter [codegen id : 6] -Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Condition : isnotnull(i_item_sk#65) +(72) Filter [codegen id : 6] +Input [6]: [i_item_sk#74, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] +Condition : isnotnull(i_item_sk#74) -(61) Exchange -Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Arguments: hashpartitioning(i_item_sk#65, 5), ENSURE_REQUIREMENTS, [id=#71] +(73) Exchange +Input [6]: [i_item_sk#74, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] +Arguments: hashpartitioning(i_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#80] -(62) Sort [codegen id : 7] -Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Arguments: [i_item_sk#65 ASC NULLS FIRST], false, 0 +(74) Sort [codegen id : 7] +Input [6]: [i_item_sk#74, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] +Arguments: [i_item_sk#74 ASC NULLS FIRST], false, 0 -(63) SortMergeJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#58] -Right keys [1]: [i_item_sk#65] +(75) SortMergeJoin [codegen id : 8] +Left keys [1]: [ss_item_sk#67] +Right keys [1]: [i_item_sk#74] Join condition: None -(64) Project [codegen id : 8] -Output [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Input [14]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +(76) Project [codegen id : 8] +Output [13]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] +Input [14]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71, i_item_sk#74, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] -(65) Exchange -Input [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Arguments: hashpartitioning(ss_ticket_number#61, ss_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#72] +(77) Exchange +Input [13]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] +Arguments: hashpartitioning(ss_ticket_number#70, ss_item_sk#67, 5), ENSURE_REQUIREMENTS, [id=#81] -(66) Sort [codegen id : 9] -Input [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Arguments: [ss_ticket_number#61 ASC NULLS FIRST, ss_item_sk#58 ASC NULLS FIRST], false, 0 +(78) Sort [codegen id : 9] +Input [13]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] +Arguments: [ss_ticket_number#70 ASC NULLS FIRST, ss_item_sk#67 ASC NULLS FIRST], false, 0 -(67) ReusedExchange [Reuses operator id: 36] -Output [2]: [sr_item_sk#73, sr_ticket_number#74] +(79) ReusedExchange [Reuses operator id: 36] +Output [2]: [sr_item_sk#82, sr_ticket_number#83] -(68) Sort [codegen id : 11] -Input [2]: [sr_item_sk#73, sr_ticket_number#74] -Arguments: [sr_ticket_number#74 ASC NULLS FIRST, sr_item_sk#73 ASC NULLS FIRST], false, 0 +(80) Sort [codegen id : 11] +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 -(69) SortMergeJoin [codegen id : 12] -Left keys [2]: [ss_ticket_number#61, ss_item_sk#58] -Right keys [2]: [sr_ticket_number#74, sr_item_sk#73] +(81) SortMergeJoin [codegen id : 12] +Left keys [2]: [ss_ticket_number#70, ss_item_sk#67] +Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] Join condition: None -(70) Project [codegen id : 12] -Output [11]: [ss_net_paid#62, s_store_name#52, s_state#53, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#56, c_last_name#57, ca_state#54] -Input [15]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, sr_item_sk#73, sr_ticket_number#74] - -(71) HashAggregate [codegen id : 12] -Input [11]: [ss_net_paid#62, s_store_name#52, s_state#53, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#56, c_last_name#57, ca_state#54] -Keys [10]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#62))] -Aggregate Attributes [1]: [sum#75] -Results [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] - -(72) Exchange -Input [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] -Arguments: hashpartitioning(c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, 5), ENSURE_REQUIREMENTS, [id=#77] - -(73) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] -Keys [10]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67] -Functions [1]: [sum(UnscaledValue(ss_net_paid#62))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#62))#78] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#62))#78,17,2) AS netpaid#40] - -(74) HashAggregate [codegen id : 13] +(82) Project [codegen id : 12] +Output [11]: [ss_net_paid#71, s_store_name#52, s_state#53, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79, c_first_name#63, c_last_name#64, ca_state#56] +Input [15]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79, sr_item_sk#82, sr_ticket_number#83] + +(83) HashAggregate [codegen id : 12] +Input [11]: [ss_net_paid#71, s_store_name#52, s_state#53, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79, c_first_name#63, c_last_name#64, ca_state#56] +Keys [10]: [c_last_name#64, c_first_name#63, s_store_name#52, ca_state#56, s_state#53, i_color#77, i_current_price#75, i_manager_id#79, i_units#78, i_size#76] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#71))] +Aggregate Attributes [1]: [sum#84] +Results [11]: [c_last_name#64, c_first_name#63, s_store_name#52, ca_state#56, s_state#53, i_color#77, i_current_price#75, i_manager_id#79, i_units#78, i_size#76, sum#85] + +(84) Exchange +Input [11]: [c_last_name#64, c_first_name#63, s_store_name#52, ca_state#56, s_state#53, i_color#77, i_current_price#75, i_manager_id#79, i_units#78, i_size#76, sum#85] +Arguments: hashpartitioning(c_last_name#64, c_first_name#63, s_store_name#52, ca_state#56, s_state#53, i_color#77, i_current_price#75, i_manager_id#79, i_units#78, i_size#76, 5), ENSURE_REQUIREMENTS, [id=#86] + +(85) HashAggregate [codegen id : 13] +Input [11]: [c_last_name#64, c_first_name#63, s_store_name#52, ca_state#56, s_state#53, i_color#77, i_current_price#75, i_manager_id#79, i_units#78, i_size#76, sum#85] +Keys [10]: [c_last_name#64, c_first_name#63, s_store_name#52, ca_state#56, s_state#53, i_color#77, i_current_price#75, i_manager_id#79, i_units#78, i_size#76] +Functions [1]: [sum(UnscaledValue(ss_net_paid#71))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#71))#87] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#71))#87,17,2) AS netpaid#40] + +(86) HashAggregate [codegen id : 13] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#79, count#80] -Results [2]: [sum#81, count#82] +Aggregate Attributes [2]: [sum#88, count#89] +Results [2]: [sum#90, count#91] -(75) Exchange -Input [2]: [sum#81, count#82] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#83] +(87) Exchange +Input [2]: [sum#90, count#91] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#92] -(76) HashAggregate [codegen id : 14] -Input [2]: [sum#81, count#82] +(88) HashAggregate [codegen id : 14] +Input [2]: [sum#90, count#91] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#84] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#84)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#85] +Aggregate Attributes [1]: [avg(netpaid#40)#93] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#93)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#94] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt index 4beebcbbe52ef..3f3511a2b92a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt @@ -35,7 +35,25 @@ WholeStageCodegen (12) Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid] BroadcastHashJoin [s_store_sk,c_customer_sk,ss_store_sk,ss_customer_sk] InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] #5 + BroadcastExchange #14 + WholeStageCodegen (3) + Project [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] + BroadcastHashJoin [ca_address_sk,upper(spark_catalog.default.customer_address.ca_country),c_current_addr_sk,c_birth_country] + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (2) + Project [s_store_sk,s_store_name,s_state,ca_address_sk,ca_state,ca_country] + BroadcastHashJoin [s_zip,ca_zip] + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #7 + Filter [ca_address_sk,ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + Filter [c_customer_sk,c_current_addr_sk,c_birth_country] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow @@ -45,7 +63,7 @@ WholeStageCodegen (12) WholeStageCodegen (7) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #14 + Exchange [i_item_sk] #16 WholeStageCodegen (6) Filter [i_item_sk] ColumnarToRow 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 db497ae40b481..1e64471e096cb 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 @@ -223,61 +223,61 @@ Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_yea Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (rn#39 + 1) AS (v1_lag.rn + 1)#40] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] (40) Exchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] (41) Sort [codegen id : 23] -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (v1_lag.rn + 1)#40 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None (43) Project [codegen id : 24] Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37] -Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] +Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] (44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Output [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] (45) Sort [codegen id : 33] -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST], false, 0 (46) Window -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] +Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Arguments: [rank(d_year#45, d_moy#46) windowspecdefinition(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#48], [i_category#41, i_brand#42, s_store_name#43, s_company_name#44], [d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (rn#49 - 1) AS (v1_lead.rn - 1)#50] -Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] +Output [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Input [8]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47, rn#48] (48) Exchange -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] -Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Arguments: hashpartitioning(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1), 5), ENSURE_REQUIREMENTS, [id=#49] (49) Sort [codegen id : 35] -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (v1_lead.rn - 1)#50 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, (rn#48 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50] +Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None (51) Project [codegen id : 36] -Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#52, sum_sales#48 AS nsum#53] -Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] +Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] +Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] (52) TakeOrderedAndProject -Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] +Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt index f522534d62c80..5f64a22717270 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] WholeStageCodegen (36) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,s_store_name,s_company_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] #8 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #8 WholeStageCodegen (22) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] #10 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #10 WholeStageCodegen (34) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter 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 0d87664dfb7a5..e7cd713d07cb4 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 @@ -223,61 +223,61 @@ Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sale Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (rn#37 + 1) AS (v1_lag.rn + 1)#38] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] (40) Exchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#38] (41) Sort [codegen id : 23] -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (v1_lag.rn + 1)#38 ASC NULLS FIRST], false, 0 +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None (43) Project [codegen id : 24] Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] (44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] (45) Sort [codegen id : 33] -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 (46) Window -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] +Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#39, i_brand#40, cc_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#45], [i_category#39, i_brand#40, cc_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (rn#46 - 1) AS (v1_lead.rn - 1)#47] -Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] +Output [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] (48) Exchange -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] -Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: hashpartitioning(i_category#39, i_brand#40, cc_name#41, (rn#45 - 1), 5), ENSURE_REQUIREMENTS, [id=#46] (49) Sort [codegen id : 35] -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (v1_lead.rn - 1)#47 ASC NULLS FIRST], false, 0 +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, (rn#45 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47] +Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None (51) Project [codegen id : 36] -Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#49, sum_sales#45 AS nsum#50] -Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] +Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] +Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] (52) TakeOrderedAndProject -Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] +Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt index f5acc760c88f3..b464f558bbc1a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] WholeStageCodegen (36) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lead.rn - 1)] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lag.rn + 1)] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,cc_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,cc_name,(v1_lag.rn + 1)] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [i_category,i_brand,cc_name,(v1_lag.rn + 1)] #8 + Exchange [i_category,i_brand,cc_name,rn] #8 WholeStageCodegen (22) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,cc_name,(v1_lead.rn - 1)] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [i_category,i_brand,cc_name,(v1_lead.rn - 1)] #10 + Exchange [i_category,i_brand,cc_name,rn] #10 WholeStageCodegen (34) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt index 67d964ee31184..fea0bcbbef17e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt @@ -350,94 +350,94 @@ Right keys [1]: [customer_id#56] Join condition: None (60) Project [codegen id : 26] -Output [6]: [customer_id#17, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#57, CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#38) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#59] +Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] (61) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +Output [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#23)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] (63) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Condition : isnotnull(ws_bill_customer_sk#60) +Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] +Condition : isnotnull(ws_bill_customer_sk#59) (64) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#63, d_year#64] +Output [2]: [d_date_sk#62, d_year#63] (65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#62] -Right keys [1]: [d_date_sk#63] +Left keys [1]: [ws_sold_date_sk#61] +Right keys [1]: [d_date_sk#62] Join condition: None (66) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64] -Input [5]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62, d_date_sk#63, d_year#64] +Output [3]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63] +Input [5]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61, d_date_sk#62, d_year#63] (67) Exchange -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64] -Arguments: hashpartitioning(ws_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63] +Arguments: hashpartitioning(ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#64] (68) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64] -Arguments: [ws_bill_customer_sk#60 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63] +Arguments: [ws_bill_customer_sk#59 ASC NULLS FIRST], false, 0 (69) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69] +Output [4]: [c_customer_sk#65, c_customer_id#66, c_first_name#67, c_last_name#68] (70) Sort [codegen id : 31] -Input [4]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69] -Arguments: [c_customer_sk#66 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#65, c_customer_id#66, c_first_name#67, c_last_name#68] +Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#60] -Right keys [1]: [c_customer_sk#66] +Left keys [1]: [ws_bill_customer_sk#59] +Right keys [1]: [c_customer_sk#65] Join condition: None (72) Project [codegen id : 32] -Output [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, ws_net_paid#61, d_year#64] -Input [7]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64, c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69] +Output [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, ws_net_paid#60, d_year#63] +Input [7]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63, c_customer_sk#65, c_customer_id#66, c_first_name#67, c_last_name#68] (73) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, ws_net_paid#61, d_year#64] -Keys [4]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#61))] -Aggregate Attributes [1]: [sum#70] -Results [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, sum#71] +Input [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, ws_net_paid#60, d_year#63] +Keys [4]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#60))] +Aggregate Attributes [1]: [sum#69] +Results [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, sum#70] (74) Exchange -Input [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, sum#71] -Arguments: hashpartitioning(c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, sum#70] +Arguments: hashpartitioning(c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, 5), ENSURE_REQUIREMENTS, [id=#71] (75) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, sum#71] -Keys [4]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64] -Functions [1]: [sum(UnscaledValue(ws_net_paid#61))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#61))#73] -Results [2]: [c_customer_id#67 AS customer_id#74, MakeDecimal(sum(UnscaledValue(ws_net_paid#61))#73,17,2) AS year_total#75] +Input [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, sum#70] +Keys [4]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63] +Functions [1]: [sum(UnscaledValue(ws_net_paid#60))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#60))#72] +Results [2]: [c_customer_id#66 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#60))#72,17,2) AS year_total#74] (76) Exchange -Input [2]: [customer_id#74, year_total#75] -Arguments: hashpartitioning(customer_id#74, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [2]: [customer_id#73, year_total#74] +Arguments: hashpartitioning(customer_id#73, 5), ENSURE_REQUIREMENTS, [id=#75] (77) Sort [codegen id : 34] -Input [2]: [customer_id#74, year_total#75] -Arguments: [customer_id#74 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#73, year_total#74] +Arguments: [customer_id#73 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#74] -Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#75) / promote_precision(year_total#57)), DecimalType(37,20), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#59) +Right keys [1]: [customer_id#73] +Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#38) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) (79) Project [codegen id : 35] Output [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] -Input [8]: [customer_id#17, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#57, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#59, customer_id#74, year_total#75] +Input [9]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57, customer_id#73, year_total#74] (80) TakeOrderedAndProject Input [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] @@ -468,7 +468,7 @@ Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2001)) AND d_year#6 IN (2001, (84) BroadcastExchange Input [2]: [d_date_sk#5, d_year#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (88) @@ -493,10 +493,10 @@ Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (20 (88) BroadcastExchange Input [2]: [d_date_sk#24, d_year#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#23 +Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#23 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt index e4c930b0c6f25..99e72fe265b6b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (35) Project [customer_id,customer_first_name,customer_last_name] - SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END] + SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (26) - Project [customer_id,customer_id,customer_first_name,customer_last_name,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) From 431e873054387fbf44b0a394bbdf37829d54ff7d Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 1 Aug 2021 08:58:27 +0800 Subject: [PATCH 05/17] Fix test error --- .../optimizer/PullOutJoinCondition.scala | 5 +- .../optimizer/PullOutJoinConditionSuite.scala | 69 ++-- .../approved-plans-v2_7/q24.sf100/explain.txt | 306 +++++++----------- .../q24.sf100/simplified.txt | 22 +- 4 files changed, 169 insertions(+), 233 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala index 7c2d6a2fbe3c8..2adeff0c1b150 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala @@ -44,7 +44,8 @@ object PullOutJoinCondition extends Rule[LogicalPlan] with JoinSelectionHelper with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(_.containsPattern(JOIN)) { - case j @ Join(left, right, _, Some(condition), _) if j.resolved => + case j @ Join(left, right, _, Some(condition), _) + if j.resolved && !j.isStreaming && !canPlanAsBroadcastHashJoin(j, conf) => val complexExpressions = splitConjunctivePredicates(condition).flatMap(_.children).flatMap { case e: Expression if !e.foldable && e.children.nonEmpty => Seq(e) case _ => Nil @@ -67,7 +68,7 @@ object PullOutJoinCondition extends Rule[LogicalPlan] } val newLeft = Project(left.output ++ leftComplexExpressionMap.values, left) val newRight = Project(right.output ++ rightComplexExpressionMap.values, right) - j.copy(left = newLeft, right = newRight, condition = Some(newCondition)) + Project(j.output, j.copy(left = newLeft, right = newRight, condition = Some(newCondition))) } else { j } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala index ba837b5fac228..e1a53d5c72388 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Coalesce, Substring, Up import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.internal.SQLConf class PullOutJoinConditionSuite extends PlanTest { @@ -39,45 +40,63 @@ class PullOutJoinConditionSuite extends PlanTest { private val y = testRelation1.subquery('y) test("Push down join condition evaluation(String expressions)") { - val joinType = Inner - Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => - val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) - .select("x.a".attr, "y.e".attr) - val correctAnswer = x.select("x.a".attr, "x.b".attr, "x.c".attr) - .join(y.select("y.d".attr, "y.e".attr, Alias(udf, udf.sql)()), - joinType, Option("x.a".attr === s"`${udf.sql}`".attr)).select("x.a".attr, "y.e".attr) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val joinType = Inner + Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => + val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) + .select("x.a".attr, "y.e".attr) + val correctAnswer = x.select("x.a".attr, "x.b".attr, "x.c".attr) + .join(y.select("y.d".attr, "y.e".attr, Alias(udf, udf.sql)()), + joinType, Option("x.a".attr === s"`${udf.sql}`".attr)).select("x.a".attr, "y.e".attr) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } } } test("Push down join condition evaluation(null expressions)") { - val joinType = Inner - val udf = Coalesce(Seq("x.b".attr, "x.c".attr)) - val originalQuery = x.join(y, joinType, Option(udf === "y.e".attr)) - .select("x.a".attr, "y.e".attr) - val correctAnswer = - x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join( - y.select("y.d".attr, "y.e".attr), - joinType, Option(s"`${udf.sql}`".attr === "y.e".attr)) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val joinType = Inner + val udf = Coalesce(Seq("x.b".attr, "x.c".attr)) + val originalQuery = x.join(y, joinType, Option(udf === "y.e".attr)) .select("x.a".attr, "y.e".attr) + val correctAnswer = + x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join( + y.select("y.d".attr, "y.e".attr), + joinType, Option(s"`${udf.sql}`".attr === "y.e".attr)) + .select("x.a".attr, "y.e".attr) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + } - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + test("Negative case: Non broadcast hash join") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000000") { + val joinType = Inner + val udf = Upper("y.d".attr) + val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) + + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) + } } test("Negative case: all children are Attributes") { - val condition = Option("x.a".attr === "y.d".attr) - val originalQuery = x.join(y, Inner, condition) - val correctAnswer = x.join(y, Inner, condition) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val condition = Option("x.a".attr === "y.d".attr) + val originalQuery = x.join(y, Inner, condition) + val correctAnswer = x.join(y, Inner, condition) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } } test("Negative case: contains Literal") { - val condition = Option("x.a".attr === "string") - val originalQuery = x.join(y, Inner, condition) - val correctAnswer = x.join(y, Inner, condition) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val condition = Option("x.a".attr === "string") + val originalQuery = x.join(y, Inner, condition) + val correctAnswer = x.join(y, Inner, condition) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } } } diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt index 717e343973617..c8cd870c62c8a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt @@ -273,229 +273,163 @@ Arguments: [c_last_name#15 ASC NULLS FIRST, c_first_name#14 ASC NULLS FIRST, s_s ===== Subqueries ===== Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#48, [id=#49] -* HashAggregate (88) -+- Exchange (87) - +- * HashAggregate (86) - +- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- * Project (82) - +- * SortMergeJoin Inner (81) - :- * Sort (78) - : +- Exchange (77) - : +- * Project (76) - : +- * SortMergeJoin Inner (75) - : :- * Sort (69) - : : +- Exchange (68) - : : +- * Project (67) - : : +- * BroadcastHashJoin Inner BuildLeft (66) - : : :- BroadcastExchange (61) - : : : +- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildLeft (59) - : : : :- BroadcastExchange (55) - : : : : +- * Project (54) - : : : : +- * BroadcastHashJoin Inner BuildLeft (53) - : : : : :- ReusedExchange (49) - : : : : +- * Filter (52) - : : : : +- * ColumnarToRow (51) - : : : : +- Scan parquet default.customer_address (50) - : : : +- * Filter (58) - : : : +- * ColumnarToRow (57) - : : : +- Scan parquet default.customer (56) - : : +- * Project (65) - : : +- * Filter (64) - : : +- * ColumnarToRow (63) - : : +- Scan parquet default.store_sales (62) - : +- * Sort (74) - : +- Exchange (73) - : +- * Filter (72) - : +- * ColumnarToRow (71) - : +- Scan parquet default.item (70) - +- * Sort (80) - +- ReusedExchange (79) - - -(49) ReusedExchange [Reuses operator id: 5] -Output [4]: [s_store_sk#51, s_store_name#52, s_state#53, s_zip#54] - -(50) Scan parquet default.customer_address -Output [4]: [ca_address_sk#55, ca_state#56, ca_zip#57, ca_country#58] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] -ReadSchema: struct - -(51) ColumnarToRow -Input [4]: [ca_address_sk#55, ca_state#56, ca_zip#57, ca_country#58] - -(52) Filter -Input [4]: [ca_address_sk#55, ca_state#56, ca_zip#57, ca_country#58] -Condition : (((isnotnull(ca_address_sk#55) AND isnotnull(ca_country#58)) AND isnotnull(ca_zip#57)) AND isnotnull(upper(ca_country#58))) - -(53) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_zip#54] -Right keys [1]: [ca_zip#57] -Join condition: None - -(54) Project [codegen id : 2] -Output [6]: [s_store_sk#51, s_store_name#52, s_state#53, ca_address_sk#55, ca_state#56, upper(ca_country#58) AS upper(spark_catalog.default.customer_address.ca_country)#59] -Input [8]: [s_store_sk#51, s_store_name#52, s_state#53, s_zip#54, ca_address_sk#55, ca_state#56, ca_zip#57, ca_country#58] - -(55) BroadcastExchange -Input [6]: [s_store_sk#51, s_store_name#52, s_state#53, ca_address_sk#55, ca_state#56, upper(spark_catalog.default.customer_address.ca_country)#59] -Arguments: HashedRelationBroadcastMode(List(input[3, int, true], input[5, string, true]),false), [id=#60] - -(56) Scan parquet default.customer -Output [5]: [c_customer_sk#61, c_current_addr_sk#62, c_first_name#63, c_last_name#64, c_birth_country#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(57) ColumnarToRow -Input [5]: [c_customer_sk#61, c_current_addr_sk#62, c_first_name#63, c_last_name#64, c_birth_country#65] - -(58) Filter -Input [5]: [c_customer_sk#61, c_current_addr_sk#62, c_first_name#63, c_last_name#64, c_birth_country#65] -Condition : ((isnotnull(c_customer_sk#61) AND isnotnull(c_current_addr_sk#62)) AND isnotnull(c_birth_country#65)) - -(59) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ca_address_sk#55, upper(spark_catalog.default.customer_address.ca_country)#59] -Right keys [2]: [c_current_addr_sk#62, c_birth_country#65] -Join condition: None - -(60) Project [codegen id : 3] -Output [7]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#56, c_customer_sk#61, c_first_name#63, c_last_name#64] -Input [11]: [s_store_sk#51, s_store_name#52, s_state#53, ca_address_sk#55, ca_state#56, upper(spark_catalog.default.customer_address.ca_country)#59, c_customer_sk#61, c_current_addr_sk#62, c_first_name#63, c_last_name#64, c_birth_country#65] - -(61) BroadcastExchange -Input [7]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#56, c_customer_sk#61, c_first_name#63, c_last_name#64] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#66] - -(62) Scan parquet default.store_sales -Output [6]: [ss_item_sk#67, ss_customer_sk#68, ss_store_sk#69, ss_ticket_number#70, ss_net_paid#71, ss_sold_date_sk#72] +* HashAggregate (76) ++- Exchange (75) + +- * HashAggregate (74) + +- * HashAggregate (73) + +- Exchange (72) + +- * HashAggregate (71) + +- * Project (70) + +- * SortMergeJoin Inner (69) + :- * Sort (66) + : +- Exchange (65) + : +- * Project (64) + : +- * SortMergeJoin Inner (63) + : :- * Sort (57) + : : +- Exchange (56) + : : +- * Project (55) + : : +- * BroadcastHashJoin Inner BuildLeft (54) + : : :- ReusedExchange (49) + : : +- * Project (53) + : : +- * Filter (52) + : : +- * ColumnarToRow (51) + : : +- Scan parquet default.store_sales (50) + : +- * Sort (62) + : +- Exchange (61) + : +- * Filter (60) + : +- * ColumnarToRow (59) + : +- Scan parquet default.item (58) + +- * Sort (68) + +- ReusedExchange (67) + + +(49) ReusedExchange [Reuses operator id: 17] +Output [7]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#54, c_customer_sk#55, c_first_name#56, c_last_name#57] + +(50) Scan parquet default.store_sales +Output [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(63) ColumnarToRow -Input [6]: [ss_item_sk#67, ss_customer_sk#68, ss_store_sk#69, ss_ticket_number#70, ss_net_paid#71, ss_sold_date_sk#72] +(51) ColumnarToRow +Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] -(64) Filter -Input [6]: [ss_item_sk#67, ss_customer_sk#68, ss_store_sk#69, ss_ticket_number#70, ss_net_paid#71, ss_sold_date_sk#72] -Condition : (((isnotnull(ss_ticket_number#70) AND isnotnull(ss_item_sk#67)) AND isnotnull(ss_store_sk#69)) AND isnotnull(ss_customer_sk#68)) +(52) Filter +Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] +Condition : (((isnotnull(ss_ticket_number#61) AND isnotnull(ss_item_sk#58)) AND isnotnull(ss_store_sk#60)) AND isnotnull(ss_customer_sk#59)) -(65) Project -Output [5]: [ss_item_sk#67, ss_customer_sk#68, ss_store_sk#69, ss_ticket_number#70, ss_net_paid#71] -Input [6]: [ss_item_sk#67, ss_customer_sk#68, ss_store_sk#69, ss_ticket_number#70, ss_net_paid#71, ss_sold_date_sk#72] +(53) Project +Output [5]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62] +Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] -(66) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [s_store_sk#51, c_customer_sk#61] -Right keys [2]: [ss_store_sk#69, ss_customer_sk#68] +(54) BroadcastHashJoin [codegen id : 4] +Left keys [2]: [s_store_sk#51, c_customer_sk#55] +Right keys [2]: [ss_store_sk#60, ss_customer_sk#59] Join condition: None -(67) Project [codegen id : 4] -Output [8]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71] -Input [12]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#56, c_customer_sk#61, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_customer_sk#68, ss_store_sk#69, ss_ticket_number#70, ss_net_paid#71] +(55) Project [codegen id : 4] +Output [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] +Input [12]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#54, c_customer_sk#55, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62] -(68) Exchange -Input [8]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71] -Arguments: hashpartitioning(ss_item_sk#67, 5), ENSURE_REQUIREMENTS, [id=#73] +(56) Exchange +Input [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] +Arguments: hashpartitioning(ss_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#64] -(69) Sort [codegen id : 5] -Input [8]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71] -Arguments: [ss_item_sk#67 ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 5] +Input [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] +Arguments: [ss_item_sk#58 ASC NULLS FIRST], false, 0 -(70) Scan parquet default.item -Output [6]: [i_item_sk#74, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] +(58) Scan parquet default.item +Output [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(71) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#74, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] +(59) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -(72) Filter [codegen id : 6] -Input [6]: [i_item_sk#74, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] -Condition : isnotnull(i_item_sk#74) +(60) Filter [codegen id : 6] +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Condition : isnotnull(i_item_sk#65) -(73) Exchange -Input [6]: [i_item_sk#74, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] -Arguments: hashpartitioning(i_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#80] +(61) Exchange +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Arguments: hashpartitioning(i_item_sk#65, 5), ENSURE_REQUIREMENTS, [id=#71] -(74) Sort [codegen id : 7] -Input [6]: [i_item_sk#74, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] -Arguments: [i_item_sk#74 ASC NULLS FIRST], false, 0 +(62) Sort [codegen id : 7] +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Arguments: [i_item_sk#65 ASC NULLS FIRST], false, 0 -(75) SortMergeJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#67] -Right keys [1]: [i_item_sk#74] +(63) SortMergeJoin [codegen id : 8] +Left keys [1]: [ss_item_sk#58] +Right keys [1]: [i_item_sk#65] Join condition: None -(76) Project [codegen id : 8] -Output [13]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] -Input [14]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71, i_item_sk#74, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] +(64) Project [codegen id : 8] +Output [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Input [14]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -(77) Exchange -Input [13]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] -Arguments: hashpartitioning(ss_ticket_number#70, ss_item_sk#67, 5), ENSURE_REQUIREMENTS, [id=#81] +(65) Exchange +Input [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Arguments: hashpartitioning(ss_ticket_number#61, ss_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#72] -(78) Sort [codegen id : 9] -Input [13]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79] -Arguments: [ss_ticket_number#70 ASC NULLS FIRST, ss_item_sk#67 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 9] +Input [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Arguments: [ss_ticket_number#61 ASC NULLS FIRST, ss_item_sk#58 ASC NULLS FIRST], false, 0 -(79) ReusedExchange [Reuses operator id: 36] -Output [2]: [sr_item_sk#82, sr_ticket_number#83] +(67) ReusedExchange [Reuses operator id: 36] +Output [2]: [sr_item_sk#73, sr_ticket_number#74] -(80) Sort [codegen id : 11] -Input [2]: [sr_item_sk#82, sr_ticket_number#83] -Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 11] +Input [2]: [sr_item_sk#73, sr_ticket_number#74] +Arguments: [sr_ticket_number#74 ASC NULLS FIRST, sr_item_sk#73 ASC NULLS FIRST], false, 0 -(81) SortMergeJoin [codegen id : 12] -Left keys [2]: [ss_ticket_number#70, ss_item_sk#67] -Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] +(69) SortMergeJoin [codegen id : 12] +Left keys [2]: [ss_ticket_number#61, ss_item_sk#58] +Right keys [2]: [sr_ticket_number#74, sr_item_sk#73] Join condition: None -(82) Project [codegen id : 12] -Output [11]: [ss_net_paid#71, s_store_name#52, s_state#53, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79, c_first_name#63, c_last_name#64, ca_state#56] -Input [15]: [s_store_name#52, s_state#53, ca_state#56, c_first_name#63, c_last_name#64, ss_item_sk#67, ss_ticket_number#70, ss_net_paid#71, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79, sr_item_sk#82, sr_ticket_number#83] - -(83) HashAggregate [codegen id : 12] -Input [11]: [ss_net_paid#71, s_store_name#52, s_state#53, i_current_price#75, i_size#76, i_color#77, i_units#78, i_manager_id#79, c_first_name#63, c_last_name#64, ca_state#56] -Keys [10]: [c_last_name#64, c_first_name#63, s_store_name#52, ca_state#56, s_state#53, i_color#77, i_current_price#75, i_manager_id#79, i_units#78, i_size#76] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#71))] -Aggregate Attributes [1]: [sum#84] -Results [11]: [c_last_name#64, c_first_name#63, s_store_name#52, ca_state#56, s_state#53, i_color#77, i_current_price#75, i_manager_id#79, i_units#78, i_size#76, sum#85] - -(84) Exchange -Input [11]: [c_last_name#64, c_first_name#63, s_store_name#52, ca_state#56, s_state#53, i_color#77, i_current_price#75, i_manager_id#79, i_units#78, i_size#76, sum#85] -Arguments: hashpartitioning(c_last_name#64, c_first_name#63, s_store_name#52, ca_state#56, s_state#53, i_color#77, i_current_price#75, i_manager_id#79, i_units#78, i_size#76, 5), ENSURE_REQUIREMENTS, [id=#86] - -(85) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#64, c_first_name#63, s_store_name#52, ca_state#56, s_state#53, i_color#77, i_current_price#75, i_manager_id#79, i_units#78, i_size#76, sum#85] -Keys [10]: [c_last_name#64, c_first_name#63, s_store_name#52, ca_state#56, s_state#53, i_color#77, i_current_price#75, i_manager_id#79, i_units#78, i_size#76] -Functions [1]: [sum(UnscaledValue(ss_net_paid#71))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#71))#87] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#71))#87,17,2) AS netpaid#40] - -(86) HashAggregate [codegen id : 13] +(70) Project [codegen id : 12] +Output [11]: [ss_net_paid#62, s_store_name#52, s_state#53, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#56, c_last_name#57, ca_state#54] +Input [15]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, sr_item_sk#73, sr_ticket_number#74] + +(71) HashAggregate [codegen id : 12] +Input [11]: [ss_net_paid#62, s_store_name#52, s_state#53, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#56, c_last_name#57, ca_state#54] +Keys [10]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#62))] +Aggregate Attributes [1]: [sum#75] +Results [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] + +(72) Exchange +Input [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] +Arguments: hashpartitioning(c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, 5), ENSURE_REQUIREMENTS, [id=#77] + +(73) HashAggregate [codegen id : 13] +Input [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] +Keys [10]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67] +Functions [1]: [sum(UnscaledValue(ss_net_paid#62))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#62))#78] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#62))#78,17,2) AS netpaid#40] + +(74) HashAggregate [codegen id : 13] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#88, count#89] -Results [2]: [sum#90, count#91] +Aggregate Attributes [2]: [sum#79, count#80] +Results [2]: [sum#81, count#82] -(87) Exchange -Input [2]: [sum#90, count#91] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#92] +(75) Exchange +Input [2]: [sum#81, count#82] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#83] -(88) HashAggregate [codegen id : 14] -Input [2]: [sum#90, count#91] +(76) HashAggregate [codegen id : 14] +Input [2]: [sum#81, count#82] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#93] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#93)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#94] +Aggregate Attributes [1]: [avg(netpaid#40)#84] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#84)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#85] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt index 3f3511a2b92a7..4beebcbbe52ef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt @@ -35,25 +35,7 @@ WholeStageCodegen (12) Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid] BroadcastHashJoin [s_store_sk,c_customer_sk,ss_store_sk,ss_customer_sk] InputAdapter - BroadcastExchange #14 - WholeStageCodegen (3) - Project [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] - BroadcastHashJoin [ca_address_sk,upper(spark_catalog.default.customer_address.ca_country),c_current_addr_sk,c_birth_country] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (2) - Project [s_store_sk,s_store_name,s_state,ca_address_sk,ca_state,ca_country] - BroadcastHashJoin [s_zip,ca_zip] - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #7 - Filter [ca_address_sk,ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] - Filter [c_customer_sk,c_current_addr_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + ReusedExchange [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] #5 Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow @@ -63,7 +45,7 @@ WholeStageCodegen (12) WholeStageCodegen (7) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #16 + Exchange [i_item_sk] #14 WholeStageCodegen (6) Filter [i_item_sk] ColumnarToRow From 990a1b4c7c55510c237a670c40493599f70bf979 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 1 Aug 2021 15:36:02 +0800 Subject: [PATCH 06/17] Move rule to SparkOptimizer --- .../sql/catalyst/optimizer/Optimizer.scala | 1 - .../spark/sql/execution/SparkOptimizer.scala | 5 +- .../approved-plans-v1_4/q11.sf100/explain.txt | 88 +++--- .../q11.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q15.sf100/explain.txt | 100 +++---- .../q15.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q4.sf100/explain.txt | 262 +++++++++--------- .../q4.sf100/simplified.txt | 6 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 48 ++-- .../q47.sf100/simplified.txt | 12 +- .../approved-plans-v1_4/q57.sf100/explain.txt | 48 ++-- .../q57.sf100/simplified.txt | 12 +- .../approved-plans-v1_4/q8.sf100/explain.txt | 162 +++++------ .../q8.sf100/simplified.txt | 12 +- .../approved-plans-v2_7/q11.sf100/explain.txt | 88 +++--- .../q11.sf100/simplified.txt | 4 +- .../approved-plans-v2_7/q47.sf100/explain.txt | 48 ++-- .../q47.sf100/simplified.txt | 12 +- .../approved-plans-v2_7/q57.sf100/explain.txt | 48 ++-- .../q57.sf100/simplified.txt | 12 +- .../approved-plans-v2_7/q74.sf100/explain.txt | 88 +++--- .../q74.sf100/simplified.txt | 4 +- 22 files changed, 535 insertions(+), 533 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index dcc6da9afd6c1..bcf18b2140492 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 @@ -151,7 +151,6 @@ abstract class Optimizer(catalogManager: CatalogManager) ReplaceExpressions, RewriteNonCorrelatedExists, PullOutGroupingExpressions, - PullOutJoinCondition, ComputeCurrentTime, ReplaceCurrentLike(catalogManager)) :: ////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index dde5dc2be0556..01884006f10ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -48,7 +48,10 @@ class SparkOptimizer( PushDownPredicates) :+ Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, - PruneFilters)) ++ + PruneFilters) :+ + Batch("Pull out join condition", Once, + PullOutJoinCondition, + CollapseProject)) ++ postHocOptimizationBatches :+ Batch("Extract Python UDFs", Once, ExtractPythonUDFFromJoinCondition, diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt index fa8839240f91e..fcc62c8dea526 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt @@ -355,94 +355,94 @@ Right keys [1]: [customer_id#70] Join condition: None (61) Project [codegen id : 26] -Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#71] +Output [4]: [customer_id#22, customer_preferred_cust_flag#46, year_total#71, CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#73] Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, customer_id#70, year_total#71] (62) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] +Output [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#76), dynamicpruningexpression(ws_sold_date_sk#76 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_sold_date_sk#77 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (63) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] +Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77] (64) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] -Condition : isnotnull(ws_bill_customer_sk#73) +Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77] +Condition : isnotnull(ws_bill_customer_sk#74) (65) ReusedExchange [Reuses operator id: 89] -Output [2]: [d_date_sk#77, d_year#78] +Output [2]: [d_date_sk#78, d_year#79] (66) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#76] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [ws_sold_date_sk#77] +Right keys [1]: [d_date_sk#78] Join condition: None (67) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Input [6]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76, d_date_sk#77, d_year#78] +Output [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77, d_date_sk#78, d_year#79] (68) Exchange -Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Arguments: hashpartitioning(ws_bill_customer_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] +Arguments: hashpartitioning(ws_bill_customer_sk#74, 5), ENSURE_REQUIREMENTS, [id=#80] (69) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Arguments: [ws_bill_customer_sk#73 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] +Arguments: [ws_bill_customer_sk#74 ASC NULLS FIRST], false, 0 (70) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] +Output [8]: [c_customer_sk#81, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88] (71) Sort [codegen id : 31] -Input [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] -Arguments: [c_customer_sk#80 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#81, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88] +Arguments: [c_customer_sk#81 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#73] -Right keys [1]: [c_customer_sk#80] +Left keys [1]: [ws_bill_customer_sk#74] +Right keys [1]: [c_customer_sk#81] Join condition: None (73) Project [codegen id : 32] -Output [10]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Input [12]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78, c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] +Output [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] +Input [12]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79, c_customer_sk#81, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88] (74) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Keys [8]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#88] -Results [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] +Input [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] +Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#89] +Results [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, sum#90] (75) Exchange -Input [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] -Arguments: hashpartitioning(c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, sum#90] +Arguments: hashpartitioning(c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, 5), ENSURE_REQUIREMENTS, [id=#91] (76) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] -Keys [8]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))#91] -Results [2]: [c_customer_id#81 AS customer_id#92, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))#91,18,2) AS year_total#93] +Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, sum#90] +Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))#92] +Results [2]: [c_customer_id#82 AS customer_id#93, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))#92,18,2) AS year_total#94] (77) Exchange -Input [2]: [customer_id#92, year_total#93] -Arguments: hashpartitioning(customer_id#92, 5), ENSURE_REQUIREMENTS, [id=#94] +Input [2]: [customer_id#93, year_total#94] +Arguments: hashpartitioning(customer_id#93, 5), ENSURE_REQUIREMENTS, [id=#95] (78) Sort [codegen id : 34] -Input [2]: [customer_id#92, year_total#93] -Arguments: [customer_id#92 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#93, year_total#94] +Arguments: [customer_id#93 ASC NULLS FIRST], false, 0 (79) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#92] -Join condition: (CASE WHEN (year_total#71 > 0.00) THEN CheckOverflow((promote_precision(year_total#93) / promote_precision(year_total#71)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END) +Right keys [1]: [customer_id#93] +Join condition: (CASE WHEN (year_total#71 > 0.00) THEN CheckOverflow((promote_precision(year_total#94) / promote_precision(year_total#71)), DecimalType(38,20), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#73) (80) Project [codegen id : 35] Output [1]: [customer_preferred_cust_flag#46] -Input [7]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#71, customer_id#92, year_total#93] +Input [6]: [customer_id#22, customer_preferred_cust_flag#46, year_total#71, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#73, customer_id#93, year_total#94] (81) TakeOrderedAndProject Input [1]: [customer_preferred_cust_flag#46] @@ -473,7 +473,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2001)) AND isnotnull(d_date_sk (85) BroadcastExchange Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (89) @@ -498,10 +498,10 @@ Condition : ((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND isnotnull(d_date_ (89) BroadcastExchange Input [2]: [d_date_sk#30, d_year#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#97] Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#76 IN dynamicpruning#29 +Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#29 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt index 889fc666bd810..7162fb28da5ce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (35) Project [customer_preferred_cust_flag] - SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END] InputAdapter WholeStageCodegen (26) - Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] + Project [customer_id,customer_preferred_cust_flag,year_total,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt index 55c39ed2e3288..6be8325f6af7b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt @@ -53,108 +53,108 @@ Right keys [1]: [d_date_sk#5] Join condition: None (6) Project [codegen id : 2] -Output [2]: [cs_bill_customer_sk#1, cs_sales_price#2] +Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, (cs_sales_price#2 > 500.00) AS (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6] Input [4]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, d_date_sk#5] (7) Exchange -Input [2]: [cs_bill_customer_sk#1, cs_sales_price#2] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#6] +Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#7] (8) Sort [codegen id : 3] -Input [2]: [cs_bill_customer_sk#1, cs_sales_price#2] +Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (9) Scan parquet default.customer -Output [2]: [c_customer_sk#7, c_current_addr_sk#8] +Output [2]: [c_customer_sk#8, c_current_addr_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 4] -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] +Input [2]: [c_customer_sk#8, c_current_addr_sk#9] (11) Filter [codegen id : 4] -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) +Input [2]: [c_customer_sk#8, c_current_addr_sk#9] +Condition : (isnotnull(c_customer_sk#8) AND isnotnull(c_current_addr_sk#9)) (12) Exchange -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: hashpartitioning(c_current_addr_sk#8, 5), ENSURE_REQUIREMENTS, [id=#9] +Input [2]: [c_customer_sk#8, c_current_addr_sk#9] +Arguments: hashpartitioning(c_current_addr_sk#9, 5), ENSURE_REQUIREMENTS, [id=#10] (13) Sort [codegen id : 5] -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [c_current_addr_sk#8 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#8, c_current_addr_sk#9] +Arguments: [c_current_addr_sk#9 ASC NULLS FIRST], false, 0 (14) Scan parquet default.customer_address -Output [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] +Output [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 6] -Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] +Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] (16) Filter [codegen id : 6] -Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] -Condition : isnotnull(ca_address_sk#10) +Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] +Condition : isnotnull(ca_address_sk#11) (17) Exchange -Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] -Arguments: hashpartitioning(ca_address_sk#10, 5), ENSURE_REQUIREMENTS, [id=#13] +Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] +Arguments: hashpartitioning(ca_address_sk#11, 5), ENSURE_REQUIREMENTS, [id=#14] (18) Sort [codegen id : 7] -Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] -Arguments: [ca_address_sk#10 ASC NULLS FIRST], false, 0 +Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] +Arguments: [ca_address_sk#11 ASC NULLS FIRST], false, 0 (19) SortMergeJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#8] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [c_current_addr_sk#9] +Right keys [1]: [ca_address_sk#11] Join condition: None (20) Project [codegen id : 8] -Output [3]: [c_customer_sk#7, ca_state#11, ca_zip#12] -Input [5]: [c_customer_sk#7, c_current_addr_sk#8, ca_address_sk#10, ca_state#11, ca_zip#12] +Output [3]: [c_customer_sk#8, ca_zip#13, (substr(ca_zip#13, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#12 IN (CA,WA,GA)) AS ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] +Input [5]: [c_customer_sk#8, c_current_addr_sk#9, ca_address_sk#11, ca_state#12, ca_zip#13] (21) Exchange -Input [3]: [c_customer_sk#7, ca_state#11, ca_zip#12] -Arguments: hashpartitioning(c_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [3]: [c_customer_sk#8, ca_zip#13, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] +Arguments: hashpartitioning(c_customer_sk#8, 5), ENSURE_REQUIREMENTS, [id=#16] (22) Sort [codegen id : 9] -Input [3]: [c_customer_sk#7, ca_state#11, ca_zip#12] -Arguments: [c_customer_sk#7 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#8, ca_zip#13, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] +Arguments: [c_customer_sk#8 ASC NULLS FIRST], false, 0 (23) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#7] -Join condition: ((substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#11 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) +Right keys [1]: [c_customer_sk#8] +Join condition: (((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15 OR (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6) (24) Project [codegen id : 10] -Output [2]: [cs_sales_price#2, ca_zip#12] -Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, c_customer_sk#7, ca_state#11, ca_zip#12] +Output [2]: [cs_sales_price#2, ca_zip#13] +Input [6]: [cs_bill_customer_sk#1, cs_sales_price#2, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6, c_customer_sk#8, ca_zip#13, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] (25) HashAggregate [codegen id : 10] -Input [2]: [cs_sales_price#2, ca_zip#12] -Keys [1]: [ca_zip#12] +Input [2]: [cs_sales_price#2, ca_zip#13] +Keys [1]: [ca_zip#13] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [ca_zip#12, sum#16] +Aggregate Attributes [1]: [sum#17] +Results [2]: [ca_zip#13, sum#18] (26) Exchange -Input [2]: [ca_zip#12, sum#16] -Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [2]: [ca_zip#13, sum#18] +Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, [id=#19] (27) HashAggregate [codegen id : 11] -Input [2]: [ca_zip#12, sum#16] -Keys [1]: [ca_zip#12] +Input [2]: [ca_zip#13, sum#18] +Keys [1]: [ca_zip#13] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#18] -Results [2]: [ca_zip#12, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#18,17,2) AS sum(cs_sales_price)#19] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#20] +Results [2]: [ca_zip#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#20,17,2) AS sum(cs_sales_price)#21] (28) TakeOrderedAndProject -Input [2]: [ca_zip#12, sum(cs_sales_price)#19] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST], [ca_zip#12, sum(cs_sales_price)#19] +Input [2]: [ca_zip#13, sum(cs_sales_price)#21] +Arguments: 100, [ca_zip#13 ASC NULLS FIRST], [ca_zip#13, sum(cs_sales_price)#21] ===== Subqueries ===== @@ -167,25 +167,25 @@ BroadcastExchange (33) (29) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#20, d_qoy#21] +Output [3]: [d_date_sk#5, d_year#22, d_qoy#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#20, d_qoy#21] +Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] (31) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#20, d_qoy#21] -Condition : ((((isnotnull(d_qoy#21) AND isnotnull(d_year#20)) AND (d_qoy#21 = 2)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] +Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 2)) AND (d_year#22 = 2001)) AND isnotnull(d_date_sk#5)) (32) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#20, d_qoy#21] +Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] (33) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt index 8105f67555c09..fee4d9431c8f2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (10) HashAggregate [ca_zip,cs_sales_price] [sum,sum] Project [cs_sales_price,ca_zip] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk,ca_zip,ca_state,cs_sales_price] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk,((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA'))),(spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)] InputAdapter WholeStageCodegen (3) Sort [cs_bill_customer_sk] @@ -35,7 +35,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (8) - Project [c_customer_sk,ca_state,ca_zip] + Project [c_customer_sk,ca_zip,ca_state] SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (5) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt index 4e4dd9555c718..107d5fb85a415 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt @@ -390,274 +390,274 @@ Right keys [1]: [customer_id#87] Join condition: None (60) Project [codegen id : 26] -Output [11]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#88] +Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#60) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#90] Input [12]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, customer_id#87, year_total#88] (61) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] +Output [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#95), dynamicpruningexpression(cs_sold_date_sk#95 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(cs_sold_date_sk#96), dynamicpruningexpression(cs_sold_date_sk#96 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 28] -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] +Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96] (63) Filter [codegen id : 28] -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Condition : isnotnull(cs_bill_customer_sk#90) +Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96] +Condition : isnotnull(cs_bill_customer_sk#91) (64) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#96, d_year#97] +Output [2]: [d_date_sk#97, d_year#98] (65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#95] -Right keys [1]: [d_date_sk#96] +Left keys [1]: [cs_sold_date_sk#96] +Right keys [1]: [d_date_sk#97] Join condition: None (66) Project [codegen id : 28] -Output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] -Input [8]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95, d_date_sk#96, d_year#97] +Output [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] +Input [8]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96, d_date_sk#97, d_year#98] (67) Exchange -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] -Arguments: hashpartitioning(cs_bill_customer_sk#90, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] +Arguments: hashpartitioning(cs_bill_customer_sk#91, 5), ENSURE_REQUIREMENTS, [id=#99] (68) Sort [codegen id : 29] -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] -Arguments: [cs_bill_customer_sk#90 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] +Arguments: [cs_bill_customer_sk#91 ASC NULLS FIRST], false, 0 (69) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] +Output [8]: [c_customer_sk#100, c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107] (70) Sort [codegen id : 31] -Input [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] -Arguments: [c_customer_sk#99 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#100, c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107] +Arguments: [c_customer_sk#100 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 32] -Left keys [1]: [cs_bill_customer_sk#90] -Right keys [1]: [c_customer_sk#99] +Left keys [1]: [cs_bill_customer_sk#91] +Right keys [1]: [c_customer_sk#100] Join condition: None (72) Project [codegen id : 32] -Output [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] -Input [14]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97, c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] +Output [12]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] +Input [14]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98, c_customer_sk#100, c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107] (73) HashAggregate [codegen id : 32] -Input [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] -Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#107, isEmpty#108] -Results [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, sum#109, isEmpty#110] +Input [12]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] +Keys [8]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#108, isEmpty#109] +Results [10]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, sum#110, isEmpty#111] (74) Exchange -Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, sum#109, isEmpty#110] -Arguments: hashpartitioning(c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, 5), ENSURE_REQUIREMENTS, [id=#111] +Input [10]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, sum#110, isEmpty#111] +Arguments: hashpartitioning(c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, 5), ENSURE_REQUIREMENTS, [id=#112] (75) HashAggregate [codegen id : 33] -Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, sum#109, isEmpty#110] -Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112] -Results [2]: [c_customer_id#100 AS customer_id#113, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112 AS year_total#114] +Input [10]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, sum#110, isEmpty#111] +Keys [8]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#113] +Results [2]: [c_customer_id#101 AS customer_id#114, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#113 AS year_total#115] (76) Exchange -Input [2]: [customer_id#113, year_total#114] -Arguments: hashpartitioning(customer_id#113, 5), ENSURE_REQUIREMENTS, [id=#115] +Input [2]: [customer_id#114, year_total#115] +Arguments: hashpartitioning(customer_id#114, 5), ENSURE_REQUIREMENTS, [id=#116] (77) Sort [codegen id : 34] -Input [2]: [customer_id#113, year_total#114] -Arguments: [customer_id#113 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#114, year_total#115] +Arguments: [customer_id#114 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#113] -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#60) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#114] +Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#115) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#90) (79) Project [codegen id : 35] -Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114] -Input [13]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#88, customer_id#113, year_total#114] +Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#115] +Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#90, customer_id#114, year_total#115] (80) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121] +Output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#122), dynamicpruningexpression(ws_sold_date_sk#122 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 37] -Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121] +Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] (82) Filter [codegen id : 37] -Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121] -Condition : isnotnull(ws_bill_customer_sk#116) +Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] +Condition : isnotnull(ws_bill_customer_sk#117) (83) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#122, d_year#123] +Output [2]: [d_date_sk#123, d_year#124] (84) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_sold_date_sk#121] -Right keys [1]: [d_date_sk#122] +Left keys [1]: [ws_sold_date_sk#122] +Right keys [1]: [d_date_sk#123] Join condition: None (85) Project [codegen id : 37] -Output [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] -Input [8]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121, d_date_sk#122, d_year#123] +Output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] +Input [8]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122, d_date_sk#123, d_year#124] (86) Exchange -Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] -Arguments: hashpartitioning(ws_bill_customer_sk#116, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] +Arguments: hashpartitioning(ws_bill_customer_sk#117, 5), ENSURE_REQUIREMENTS, [id=#125] (87) Sort [codegen id : 38] -Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] -Arguments: [ws_bill_customer_sk#116 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] +Arguments: [ws_bill_customer_sk#117 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +Output [8]: [c_customer_sk#126, c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133] (89) Sort [codegen id : 40] -Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] -Arguments: [c_customer_sk#125 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#126, c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133] +Arguments: [c_customer_sk#126 ASC NULLS FIRST], false, 0 (90) SortMergeJoin [codegen id : 41] -Left keys [1]: [ws_bill_customer_sk#116] -Right keys [1]: [c_customer_sk#125] +Left keys [1]: [ws_bill_customer_sk#117] +Right keys [1]: [c_customer_sk#126] Join condition: None (91) Project [codegen id : 41] -Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] -Input [14]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123, c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +Output [12]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] +Input [14]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124, c_customer_sk#126, c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133] (92) HashAggregate [codegen id : 41] -Input [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] -Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#133, isEmpty#134] -Results [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, sum#135, isEmpty#136] +Input [12]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] +Keys [8]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#134, isEmpty#135] +Results [10]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, sum#136, isEmpty#137] (93) Exchange -Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, sum#135, isEmpty#136] -Arguments: hashpartitioning(c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [10]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, sum#136, isEmpty#137] +Arguments: hashpartitioning(c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, 5), ENSURE_REQUIREMENTS, [id=#138] (94) HashAggregate [codegen id : 42] -Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, sum#135, isEmpty#136] -Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138] -Results [2]: [c_customer_id#126 AS customer_id#139, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138 AS year_total#140] +Input [10]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, sum#136, isEmpty#137] +Keys [8]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#139] +Results [2]: [c_customer_id#127 AS customer_id#140, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#139 AS year_total#141] (95) Filter [codegen id : 42] -Input [2]: [customer_id#139, year_total#140] -Condition : (isnotnull(year_total#140) AND (year_total#140 > 0.000000)) +Input [2]: [customer_id#140, year_total#141] +Condition : (isnotnull(year_total#141) AND (year_total#141 > 0.000000)) (96) Project [codegen id : 42] -Output [2]: [customer_id#139 AS customer_id#141, year_total#140 AS year_total#142] -Input [2]: [customer_id#139, year_total#140] +Output [2]: [customer_id#140 AS customer_id#142, year_total#141 AS year_total#143] +Input [2]: [customer_id#140, year_total#141] (97) Exchange -Input [2]: [customer_id#141, year_total#142] -Arguments: hashpartitioning(customer_id#141, 5), ENSURE_REQUIREMENTS, [id=#143] +Input [2]: [customer_id#142, year_total#143] +Arguments: hashpartitioning(customer_id#142, 5), ENSURE_REQUIREMENTS, [id=#144] (98) Sort [codegen id : 43] -Input [2]: [customer_id#141, year_total#142] -Arguments: [customer_id#141 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#142, year_total#143] +Arguments: [customer_id#142 ASC NULLS FIRST], false, 0 (99) SortMergeJoin [codegen id : 44] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#141] +Right keys [1]: [customer_id#142] Join condition: None (100) Project [codegen id : 44] -Output [11]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114, year_total#142] -Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114, customer_id#141, year_total#142] +Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#143, CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#115) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END AS CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#145] +Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#115, customer_id#142, year_total#143] (101) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] +Output [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#149), dynamicpruningexpression(ws_sold_date_sk#149 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(ws_sold_date_sk#151), dynamicpruningexpression(ws_sold_date_sk#151 IN dynamicpruning#35)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 46] -Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] +Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] (103) Filter [codegen id : 46] -Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] -Condition : isnotnull(ws_bill_customer_sk#144) +Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] +Condition : isnotnull(ws_bill_customer_sk#146) (104) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#150, d_year#151] +Output [2]: [d_date_sk#152, d_year#153] (105) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [ws_sold_date_sk#149] -Right keys [1]: [d_date_sk#150] +Left keys [1]: [ws_sold_date_sk#151] +Right keys [1]: [d_date_sk#152] Join condition: None (106) Project [codegen id : 46] -Output [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] -Input [8]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149, d_date_sk#150, d_year#151] +Output [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Input [8]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151, d_date_sk#152, d_year#153] (107) Exchange -Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] -Arguments: hashpartitioning(ws_bill_customer_sk#144, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Arguments: hashpartitioning(ws_bill_customer_sk#146, 5), ENSURE_REQUIREMENTS, [id=#154] (108) Sort [codegen id : 47] -Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] -Arguments: [ws_bill_customer_sk#144 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Arguments: [ws_bill_customer_sk#146 ASC NULLS FIRST], false, 0 (109) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#153, c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160] +Output [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] (110) Sort [codegen id : 49] -Input [8]: [c_customer_sk#153, c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160] -Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] +Arguments: [c_customer_sk#155 ASC NULLS FIRST], false, 0 (111) SortMergeJoin [codegen id : 50] -Left keys [1]: [ws_bill_customer_sk#144] -Right keys [1]: [c_customer_sk#153] +Left keys [1]: [ws_bill_customer_sk#146] +Right keys [1]: [c_customer_sk#155] Join condition: None (112) Project [codegen id : 50] -Output [12]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] -Input [14]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151, c_customer_sk#153, c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160] +Output [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Input [14]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153, c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] (113) HashAggregate [codegen id : 50] -Input [12]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] -Keys [8]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#161, isEmpty#162] -Results [10]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, sum#163, isEmpty#164] +Input [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#163, isEmpty#164] +Results [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] (114) Exchange -Input [10]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, sum#163, isEmpty#164] -Arguments: hashpartitioning(c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, 5), ENSURE_REQUIREMENTS, [id=#165] +Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Arguments: hashpartitioning(c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, 5), ENSURE_REQUIREMENTS, [id=#167] (115) HashAggregate [codegen id : 51] -Input [10]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, sum#163, isEmpty#164] -Keys [8]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166] -Results [2]: [c_customer_id#154 AS customer_id#167, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166 AS year_total#168] +Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#168] +Results [2]: [c_customer_id#156 AS customer_id#169, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#168 AS year_total#170] (116) Exchange -Input [2]: [customer_id#167, year_total#168] -Arguments: hashpartitioning(customer_id#167, 5), ENSURE_REQUIREMENTS, [id=#169] +Input [2]: [customer_id#169, year_total#170] +Arguments: hashpartitioning(customer_id#169, 5), ENSURE_REQUIREMENTS, [id=#171] (117) Sort [codegen id : 52] -Input [2]: [customer_id#167, year_total#168] -Arguments: [customer_id#167 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#169, year_total#170] +Arguments: [customer_id#169 ASC NULLS FIRST], false, 0 (118) SortMergeJoin [codegen id : 53] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#167] -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#142 > 0.000000) THEN CheckOverflow((promote_precision(year_total#168) / promote_precision(year_total#142)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#169] +Join condition: (CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#145 > CASE WHEN (year_total#143 > 0.000000) THEN CheckOverflow((promote_precision(year_total#170) / promote_precision(year_total#143)), DecimalType(38,14), true) ELSE null END) (119) Project [codegen id : 53] Output [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] -Input [13]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114, year_total#142, customer_id#167, year_total#168] +Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#143, CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#145, customer_id#169, year_total#170] (120) TakeOrderedAndProject Input [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] @@ -688,7 +688,7 @@ Condition : ((isnotnull(d_year#9) AND (d_year#9 = 2001)) AND isnotnull(d_date_sk (124) BroadcastExchange Input [2]: [d_date_sk#8, d_year#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#170] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#172] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#34 IN dynamicpruning#35 BroadcastExchange (128) @@ -713,14 +713,14 @@ Condition : ((isnotnull(d_year#37) AND (d_year#37 = 2002)) AND isnotnull(d_date_ (128) BroadcastExchange Input [2]: [d_date_sk#36, d_year#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#171] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#173] Subquery:3 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#7 -Subquery:4 Hosting operator id = 61 Hosting Expression = cs_sold_date_sk#95 IN dynamicpruning#35 +Subquery:4 Hosting operator id = 61 Hosting Expression = cs_sold_date_sk#96 IN dynamicpruning#35 -Subquery:5 Hosting operator id = 80 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#7 +Subquery:5 Hosting operator id = 80 Hosting Expression = ws_sold_date_sk#122 IN dynamicpruning#7 -Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#149 IN dynamicpruning#35 +Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#151 IN dynamicpruning#35 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt index d7e0a660bab5b..eab288e1d9891 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] WholeStageCodegen (53) Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + SortMergeJoin [customer_id,customer_id,CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END,year_total,year_total] InputAdapter WholeStageCodegen (44) Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] @@ -9,10 +9,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter WholeStageCodegen (35) Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END] InputAdapter WholeStageCodegen (26) - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] + Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) 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 686036a835e25..a612378242f1e 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 @@ -223,61 +223,61 @@ Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_yea Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (rn#39 + 1) AS (v1_lag.rn + 1)#40] Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] (40) Exchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40, 5), ENSURE_REQUIREMENTS, [id=#41] (41) Sort [codegen id : 23] -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] +Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (v1_lag.rn + 1)#40 ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40] Join condition: None (43) Project [codegen id : 24] Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37] -Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] (44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] (45) Sort [codegen id : 33] -Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 (46) Window -Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] -Arguments: [rank(d_year#45, d_moy#46) windowspecdefinition(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#48], [i_category#41, i_brand#42, s_store_name#43, s_company_name#44], [d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST] +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Input [8]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47, rn#48] +Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (rn#49 - 1) AS (v1_lead.rn - 1)#50] +Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] (48) Exchange -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: hashpartitioning(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1), 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] +Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50, 5), ENSURE_REQUIREMENTS, [id=#51] (49) Sort [codegen id : 35] -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, (rn#48 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (v1_lead.rn - 1)#50 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] +Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50] Join condition: None (51) Project [codegen id : 36] -Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] -Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#52, sum_sales#48 AS nsum#53] +Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] (52) TakeOrderedAndProject -Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] +Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt index 07c75d91ca3cf..5c9d3bb5049c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] WholeStageCodegen (36) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,s_store_name,s_company_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,s_store_name,s_company_name,rn] + Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #8 + Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] #8 WholeStageCodegen (22) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,s_store_name,s_company_name,rn] + Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #10 + Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] #10 WholeStageCodegen (34) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter 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 0f22f27fed99b..b7ba9e2c06424 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 @@ -223,61 +223,61 @@ Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sale Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (rn#37 + 1) AS (v1_lag.rn + 1)#38] Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] (40) Exchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38, 5), ENSURE_REQUIREMENTS, [id=#39] (41) Sort [codegen id : 23] -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (v1_lag.rn + 1)#38 ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38] Join condition: None (43) Project [codegen id : 24] Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] (44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] (45) Sort [codegen id : 33] -Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 (46) Window -Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#39, i_brand#40, cc_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#45], [i_category#39, i_brand#40, cc_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] +Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (rn#46 - 1) AS (v1_lead.rn - 1)#47] +Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] (48) Exchange -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: hashpartitioning(i_category#39, i_brand#40, cc_name#41, (rn#45 - 1), 5), ENSURE_REQUIREMENTS, [id=#46] +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] +Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47, 5), ENSURE_REQUIREMENTS, [id=#48] (49) Sort [codegen id : 35] -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, (rn#45 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (v1_lead.rn - 1)#47 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] +Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47] Join condition: None (51) Project [codegen id : 36] -Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] -Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#49, sum_sales#45 AS nsum#50] +Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] (52) TakeOrderedAndProject -Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt index 3bf10f82e6a88..f3ef99b986f71 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] WholeStageCodegen (36) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lead.rn - 1)] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lag.rn + 1)] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,cc_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,cc_name,rn] + Sort [i_category,i_brand,cc_name,(v1_lag.rn + 1)] InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #8 + Exchange [i_category,i_brand,cc_name,(v1_lag.rn + 1)] #8 WholeStageCodegen (22) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,cc_name,rn] + Sort [i_category,i_brand,cc_name,(v1_lead.rn - 1)] InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #10 + Exchange [i_category,i_brand,cc_name,(v1_lead.rn - 1)] #10 WholeStageCodegen (34) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt index e8c77e7de05eb..b6b08c52ef055 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt @@ -101,182 +101,182 @@ Right keys [1]: [s_store_sk#6] Join condition: None (12) Project [codegen id : 3] -Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] +Output [3]: [ss_net_profit#2, s_store_name#7, substr(s_zip#8, 1, 2) AS substr(spark_catalog.default.store.s_zip, 1, 2)#10] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#8] (13) Exchange -Input [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] -Arguments: hashpartitioning(substr(s_zip#8, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#10] +Input [3]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10] +Arguments: hashpartitioning(substr(spark_catalog.default.store.s_zip, 1, 2)#10, 5), ENSURE_REQUIREMENTS, [id=#11] (14) Sort [codegen id : 4] -Input [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] -Arguments: [substr(s_zip#8, 1, 2) ASC NULLS FIRST], false, 0 +Input [3]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10] +Arguments: [substr(spark_catalog.default.store.s_zip, 1, 2)#10 ASC NULLS FIRST], false, 0 (15) Scan parquet default.customer_address -Output [1]: [ca_zip#11] +Output [1]: [ca_zip#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] ReadSchema: struct (16) ColumnarToRow [codegen id : 11] -Input [1]: [ca_zip#11] +Input [1]: [ca_zip#12] (17) Filter [codegen id : 11] -Input [1]: [ca_zip#11] -Condition : (substr(ca_zip#11, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#11, 1, 5))) +Input [1]: [ca_zip#12] +Condition : (substr(ca_zip#12, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#12, 1, 5))) (18) Scan parquet default.customer_address -Output [2]: [ca_address_sk#12, ca_zip#13] +Output [2]: [ca_address_sk#13, ca_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#12, ca_zip#13] +Input [2]: [ca_address_sk#13, ca_zip#14] (20) Filter [codegen id : 5] -Input [2]: [ca_address_sk#12, ca_zip#13] -Condition : isnotnull(ca_address_sk#12) +Input [2]: [ca_address_sk#13, ca_zip#14] +Condition : isnotnull(ca_address_sk#13) (21) Exchange -Input [2]: [ca_address_sk#12, ca_zip#13] -Arguments: hashpartitioning(ca_address_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [2]: [ca_address_sk#13, ca_zip#14] +Arguments: hashpartitioning(ca_address_sk#13, 5), ENSURE_REQUIREMENTS, [id=#15] (22) Sort [codegen id : 6] -Input [2]: [ca_address_sk#12, ca_zip#13] -Arguments: [ca_address_sk#12 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#13, ca_zip#14] +Arguments: [ca_address_sk#13 ASC NULLS FIRST], false, 0 (23) Scan parquet default.customer -Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Output [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 7] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] (25) Filter [codegen id : 7] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Condition : ((isnotnull(c_preferred_cust_flag#16) AND (c_preferred_cust_flag#16 = Y)) AND isnotnull(c_current_addr_sk#15)) +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Condition : ((isnotnull(c_preferred_cust_flag#17) AND (c_preferred_cust_flag#17 = Y)) AND isnotnull(c_current_addr_sk#16)) (26) Project [codegen id : 7] -Output [1]: [c_current_addr_sk#15] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Output [1]: [c_current_addr_sk#16] +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] (27) Exchange -Input [1]: [c_current_addr_sk#15] -Arguments: hashpartitioning(c_current_addr_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [1]: [c_current_addr_sk#16] +Arguments: hashpartitioning(c_current_addr_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] (28) Sort [codegen id : 8] -Input [1]: [c_current_addr_sk#15] -Arguments: [c_current_addr_sk#15 ASC NULLS FIRST], false, 0 +Input [1]: [c_current_addr_sk#16] +Arguments: [c_current_addr_sk#16 ASC NULLS FIRST], false, 0 (29) SortMergeJoin [codegen id : 9] -Left keys [1]: [ca_address_sk#12] -Right keys [1]: [c_current_addr_sk#15] +Left keys [1]: [ca_address_sk#13] +Right keys [1]: [c_current_addr_sk#16] Join condition: None (30) Project [codegen id : 9] -Output [1]: [ca_zip#13] -Input [3]: [ca_address_sk#12, ca_zip#13, c_current_addr_sk#15] +Output [1]: [ca_zip#14] +Input [3]: [ca_address_sk#13, ca_zip#14, c_current_addr_sk#16] (31) HashAggregate [codegen id : 9] -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] +Input [1]: [ca_zip#14] +Keys [1]: [ca_zip#14] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [2]: [ca_zip#13, count#19] +Aggregate Attributes [1]: [count#19] +Results [2]: [ca_zip#14, count#20] (32) Exchange -Input [2]: [ca_zip#13, count#19] -Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [2]: [ca_zip#14, count#20] +Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [id=#21] (33) HashAggregate [codegen id : 10] -Input [2]: [ca_zip#13, count#19] -Keys [1]: [ca_zip#13] +Input [2]: [ca_zip#14, count#20] +Keys [1]: [ca_zip#14] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#21] -Results [2]: [substr(ca_zip#13, 1, 5) AS ca_zip#22, count(1)#21 AS cnt#23] +Aggregate Attributes [1]: [count(1)#22] +Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#23, count(1)#22 AS cnt#24] (34) Filter [codegen id : 10] -Input [2]: [ca_zip#22, cnt#23] -Condition : (cnt#23 > 10) +Input [2]: [ca_zip#23, cnt#24] +Condition : (cnt#24 > 10) (35) Project [codegen id : 10] -Output [1]: [ca_zip#22] -Input [2]: [ca_zip#22, cnt#23] +Output [1]: [ca_zip#23] +Input [2]: [ca_zip#23, cnt#24] (36) BroadcastExchange -Input [1]: [ca_zip#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#24] +Input [1]: [ca_zip#23] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#25] (37) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [coalesce(substr(ca_zip#11, 1, 5), ), isnull(substr(ca_zip#11, 1, 5))] -Right keys [2]: [coalesce(ca_zip#22, ), isnull(ca_zip#22)] +Left keys [2]: [coalesce(substr(ca_zip#12, 1, 5), ), isnull(substr(ca_zip#12, 1, 5))] +Right keys [2]: [coalesce(ca_zip#23, ), isnull(ca_zip#23)] Join condition: None (38) Project [codegen id : 11] -Output [1]: [substr(ca_zip#11, 1, 5) AS ca_zip#25] -Input [1]: [ca_zip#11] +Output [1]: [substr(ca_zip#12, 1, 5) AS ca_zip#26] +Input [1]: [ca_zip#12] (39) HashAggregate [codegen id : 11] -Input [1]: [ca_zip#25] -Keys [1]: [ca_zip#25] +Input [1]: [ca_zip#26] +Keys [1]: [ca_zip#26] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#25] +Results [1]: [ca_zip#26] (40) Exchange -Input [1]: [ca_zip#25] -Arguments: hashpartitioning(ca_zip#25, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [1]: [ca_zip#26] +Arguments: hashpartitioning(ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#27] (41) HashAggregate [codegen id : 12] -Input [1]: [ca_zip#25] -Keys [1]: [ca_zip#25] +Input [1]: [ca_zip#26] +Keys [1]: [ca_zip#26] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#25] +Results [1]: [substr(ca_zip#26, 1, 2) AS substr(V1.ca_zip, 1, 2)#28] (42) Exchange -Input [1]: [ca_zip#25] -Arguments: hashpartitioning(substr(ca_zip#25, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#27] +Input [1]: [substr(V1.ca_zip, 1, 2)#28] +Arguments: hashpartitioning(substr(V1.ca_zip, 1, 2)#28, 5), ENSURE_REQUIREMENTS, [id=#29] (43) Sort [codegen id : 13] -Input [1]: [ca_zip#25] -Arguments: [substr(ca_zip#25, 1, 2) ASC NULLS FIRST], false, 0 +Input [1]: [substr(V1.ca_zip, 1, 2)#28] +Arguments: [substr(V1.ca_zip, 1, 2)#28 ASC NULLS FIRST], false, 0 (44) SortMergeJoin [codegen id : 14] -Left keys [1]: [substr(s_zip#8, 1, 2)] -Right keys [1]: [substr(ca_zip#25, 1, 2)] +Left keys [1]: [substr(spark_catalog.default.store.s_zip, 1, 2)#10] +Right keys [1]: [substr(V1.ca_zip, 1, 2)#28] Join condition: None (45) Project [codegen id : 14] Output [2]: [ss_net_profit#2, s_store_name#7] -Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#8, ca_zip#25] +Input [4]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10, substr(V1.ca_zip, 1, 2)#28] (46) HashAggregate [codegen id : 14] Input [2]: [ss_net_profit#2, s_store_name#7] Keys [1]: [s_store_name#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [s_store_name#7, sum#29] +Aggregate Attributes [1]: [sum#30] +Results [2]: [s_store_name#7, sum#31] (47) Exchange -Input [2]: [s_store_name#7, sum#29] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [s_store_name#7, sum#31] +Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#32] (48) HashAggregate [codegen id : 15] -Input [2]: [s_store_name#7, sum#29] +Input [2]: [s_store_name#7, sum#31] Keys [1]: [s_store_name#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] -Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS sum(ss_net_profit)#32] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#33] +Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#33,17,2) AS sum(ss_net_profit)#34] (49) TakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#32] -Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#32] +Input [2]: [s_store_name#7, sum(ss_net_profit)#34] +Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#34] ===== Subqueries ===== @@ -289,25 +289,25 @@ BroadcastExchange (54) (50) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#33, d_qoy#34] +Output [3]: [d_date_sk#5, d_year#35, d_qoy#36] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#33, d_qoy#34] +Input [3]: [d_date_sk#5, d_year#35, d_qoy#36] (52) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#33, d_qoy#34] -Condition : ((((isnotnull(d_qoy#34) AND isnotnull(d_year#33)) AND (d_qoy#34 = 2)) AND (d_year#33 = 1998)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#5, d_year#35, d_qoy#36] +Condition : ((((isnotnull(d_qoy#36) AND isnotnull(d_year#35)) AND (d_qoy#36 = 2)) AND (d_year#35 = 1998)) AND isnotnull(d_date_sk#5)) (53) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#33, d_qoy#34] +Input [3]: [d_date_sk#5, d_year#35, d_qoy#36] (54) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt index 84ac2edd606cf..435254f45f10d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt @@ -6,12 +6,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (14) HashAggregate [s_store_name,ss_net_profit] [sum,sum] Project [ss_net_profit,s_store_name] - SortMergeJoin [s_zip,ca_zip] + SortMergeJoin [substr(spark_catalog.default.store.s_zip, 1, 2),substr(V1.ca_zip, 1, 2)] InputAdapter WholeStageCodegen (4) - Sort [s_zip] + Sort [substr(spark_catalog.default.store.s_zip, 1, 2)] InputAdapter - Exchange [s_zip] #2 + Exchange [substr(spark_catalog.default.store.s_zip, 1, 2)] #2 WholeStageCodegen (3) Project [ss_net_profit,s_store_name,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -40,11 +40,11 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] Scan parquet default.store [s_store_sk,s_store_name,s_zip] InputAdapter WholeStageCodegen (13) - Sort [ca_zip] + Sort [substr(V1.ca_zip, 1, 2)] InputAdapter - Exchange [ca_zip] #5 + Exchange [substr(V1.ca_zip, 1, 2)] #5 WholeStageCodegen (12) - HashAggregate [ca_zip] + HashAggregate [ca_zip] [substr(V1.ca_zip, 1, 2)] InputAdapter Exchange [ca_zip] #6 WholeStageCodegen (11) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt index 8e15a8b4c1e2a..84342b5469473 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt @@ -350,94 +350,94 @@ Right keys [1]: [customer_id#72] Join condition: None (60) Project [codegen id : 26] -Output [8]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, year_total#73] +Output [7]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#73, CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#75] Input [9]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, customer_id#72, year_total#73] (61) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] +Output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_sold_date_sk#78 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] +Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] (63) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] -Condition : isnotnull(ws_bill_customer_sk#75) +Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_bill_customer_sk#76) (64) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#79, d_year#80] +Output [2]: [d_date_sk#80, d_year#81] (65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#78] -Right keys [1]: [d_date_sk#79] +Left keys [1]: [ws_sold_date_sk#79] +Right keys [1]: [d_date_sk#80] Join condition: None (66) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Input [6]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78, d_date_sk#79, d_year#80] +Output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#80, d_year#81] (67) Exchange -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Arguments: hashpartitioning(ws_bill_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] +Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] (68) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Arguments: [ws_bill_customer_sk#75 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] +Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 (69) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] +Output [8]: [c_customer_sk#83, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90] (70) Sort [codegen id : 31] -Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#83, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90] +Arguments: [c_customer_sk#83 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#75] -Right keys [1]: [c_customer_sk#82] +Left keys [1]: [ws_bill_customer_sk#76] +Right keys [1]: [c_customer_sk#83] Join condition: None (72) Project [codegen id : 32] -Output [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Input [12]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80, c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] +Output [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] +Input [12]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81, c_customer_sk#83, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90] (73) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#90] -Results [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Input [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] +Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#91] +Results [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, sum#92] (74) Exchange -Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] -Arguments: hashpartitioning(c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, sum#92] +Arguments: hashpartitioning(c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, 5), ENSURE_REQUIREMENTS, [id=#93] (75) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] -Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93] -Results [2]: [c_customer_id#83 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] +Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, sum#92] +Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))#94] +Results [2]: [c_customer_id#84 AS customer_id#95, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))#94,18,2) AS year_total#96] (76) Exchange -Input [2]: [customer_id#94, year_total#95] -Arguments: hashpartitioning(customer_id#94, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [2]: [customer_id#95, year_total#96] +Arguments: hashpartitioning(customer_id#95, 5), ENSURE_REQUIREMENTS, [id=#97] (77) Sort [codegen id : 34] -Input [2]: [customer_id#94, year_total#95] -Arguments: [customer_id#94 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#95, year_total#96] +Arguments: [customer_id#95 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#94] -Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END) +Right keys [1]: [customer_id#95] +Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#96) / promote_precision(year_total#73)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#75) (79) Project [codegen id : 35] Output [4]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48] -Input [10]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, year_total#73, customer_id#94, year_total#95] +Input [9]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#73, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#75, customer_id#95, year_total#96] (80) TakeOrderedAndProject Input [4]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48] @@ -468,7 +468,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2001)) AND isnotnull(d_date_sk (84) BroadcastExchange Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#98] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (88) @@ -493,10 +493,10 @@ Condition : ((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND isnotnull(d_date_ (88) BroadcastExchange Input [2]: [d_date_sk#30, d_year#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#78 IN dynamicpruning#29 +Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#29 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt index b3f8a57ba0f5b..35e8fb614399f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] WholeStageCodegen (35) Project [customer_id,customer_first_name,customer_last_name,customer_email_address] - SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END] InputAdapter WholeStageCodegen (26) - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] + Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) 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 1e64471e096cb..db497ae40b481 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 @@ -223,61 +223,61 @@ Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_yea Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (rn#39 + 1) AS (v1_lag.rn + 1)#40] Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] (40) Exchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40, 5), ENSURE_REQUIREMENTS, [id=#41] (41) Sort [codegen id : 23] -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] +Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (v1_lag.rn + 1)#40 ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40] Join condition: None (43) Project [codegen id : 24] Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37] -Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] (44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] (45) Sort [codegen id : 33] -Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 (46) Window -Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] -Arguments: [rank(d_year#45, d_moy#46) windowspecdefinition(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#48], [i_category#41, i_brand#42, s_store_name#43, s_company_name#44], [d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST] +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Input [8]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47, rn#48] +Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (rn#49 - 1) AS (v1_lead.rn - 1)#50] +Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] (48) Exchange -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: hashpartitioning(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1), 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] +Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50, 5), ENSURE_REQUIREMENTS, [id=#51] (49) Sort [codegen id : 35] -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, (rn#48 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (v1_lead.rn - 1)#50 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] +Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50] Join condition: None (51) Project [codegen id : 36] -Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] -Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#52, sum_sales#48 AS nsum#53] +Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] (52) TakeOrderedAndProject -Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] +Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt index 5f64a22717270..f522534d62c80 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] WholeStageCodegen (36) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,s_store_name,s_company_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,s_store_name,s_company_name,rn] + Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #8 + Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] #8 WholeStageCodegen (22) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,s_store_name,s_company_name,rn] + Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #10 + Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] #10 WholeStageCodegen (34) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter 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 e7cd713d07cb4..0d87664dfb7a5 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 @@ -223,61 +223,61 @@ Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sale Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (rn#37 + 1) AS (v1_lag.rn + 1)#38] Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] (40) Exchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38, 5), ENSURE_REQUIREMENTS, [id=#39] (41) Sort [codegen id : 23] -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (v1_lag.rn + 1)#38 ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38] Join condition: None (43) Project [codegen id : 24] Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] (44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] (45) Sort [codegen id : 33] -Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 (46) Window -Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#39, i_brand#40, cc_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#45], [i_category#39, i_brand#40, cc_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] +Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (rn#46 - 1) AS (v1_lead.rn - 1)#47] +Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] (48) Exchange -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: hashpartitioning(i_category#39, i_brand#40, cc_name#41, (rn#45 - 1), 5), ENSURE_REQUIREMENTS, [id=#46] +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] +Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47, 5), ENSURE_REQUIREMENTS, [id=#48] (49) Sort [codegen id : 35] -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, (rn#45 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (v1_lead.rn - 1)#47 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] +Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47] Join condition: None (51) Project [codegen id : 36] -Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] -Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#49, sum_sales#45 AS nsum#50] +Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] (52) TakeOrderedAndProject -Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt index b464f558bbc1a..f5acc760c88f3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] WholeStageCodegen (36) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lead.rn - 1)] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lag.rn + 1)] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,cc_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,cc_name,rn] + Sort [i_category,i_brand,cc_name,(v1_lag.rn + 1)] InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #8 + Exchange [i_category,i_brand,cc_name,(v1_lag.rn + 1)] #8 WholeStageCodegen (22) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,cc_name,rn] + Sort [i_category,i_brand,cc_name,(v1_lead.rn - 1)] InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #10 + Exchange [i_category,i_brand,cc_name,(v1_lead.rn - 1)] #10 WholeStageCodegen (34) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt index fea0bcbbef17e..67d964ee31184 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt @@ -350,94 +350,94 @@ Right keys [1]: [customer_id#56] Join condition: None (60) Project [codegen id : 26] -Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] +Output [6]: [customer_id#17, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#57, CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#38) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#59] Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] (61) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] +Output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#23)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] +Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] (63) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] -Condition : isnotnull(ws_bill_customer_sk#59) +Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +Condition : isnotnull(ws_bill_customer_sk#60) (64) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#62, d_year#63] +Output [2]: [d_date_sk#63, d_year#64] (65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#61] -Right keys [1]: [d_date_sk#62] +Left keys [1]: [ws_sold_date_sk#62] +Right keys [1]: [d_date_sk#63] Join condition: None (66) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63] -Input [5]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61, d_date_sk#62, d_year#63] +Output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64] +Input [5]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62, d_date_sk#63, d_year#64] (67) Exchange -Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63] -Arguments: hashpartitioning(ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64] +Arguments: hashpartitioning(ws_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (68) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63] -Arguments: [ws_bill_customer_sk#59 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64] +Arguments: [ws_bill_customer_sk#60 ASC NULLS FIRST], false, 0 (69) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#65, c_customer_id#66, c_first_name#67, c_last_name#68] +Output [4]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69] (70) Sort [codegen id : 31] -Input [4]: [c_customer_sk#65, c_customer_id#66, c_first_name#67, c_last_name#68] -Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69] +Arguments: [c_customer_sk#66 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#59] -Right keys [1]: [c_customer_sk#65] +Left keys [1]: [ws_bill_customer_sk#60] +Right keys [1]: [c_customer_sk#66] Join condition: None (72) Project [codegen id : 32] -Output [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, ws_net_paid#60, d_year#63] -Input [7]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63, c_customer_sk#65, c_customer_id#66, c_first_name#67, c_last_name#68] +Output [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, ws_net_paid#61, d_year#64] +Input [7]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64, c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69] (73) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, ws_net_paid#60, d_year#63] -Keys [4]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#60))] -Aggregate Attributes [1]: [sum#69] -Results [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, sum#70] +Input [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, ws_net_paid#61, d_year#64] +Keys [4]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#61))] +Aggregate Attributes [1]: [sum#70] +Results [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, sum#71] (74) Exchange -Input [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, sum#70] -Arguments: hashpartitioning(c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, sum#71] +Arguments: hashpartitioning(c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, 5), ENSURE_REQUIREMENTS, [id=#72] (75) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, sum#70] -Keys [4]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63] -Functions [1]: [sum(UnscaledValue(ws_net_paid#60))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#60))#72] -Results [2]: [c_customer_id#66 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#60))#72,17,2) AS year_total#74] +Input [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, sum#71] +Keys [4]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64] +Functions [1]: [sum(UnscaledValue(ws_net_paid#61))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#61))#73] +Results [2]: [c_customer_id#67 AS customer_id#74, MakeDecimal(sum(UnscaledValue(ws_net_paid#61))#73,17,2) AS year_total#75] (76) Exchange -Input [2]: [customer_id#73, year_total#74] -Arguments: hashpartitioning(customer_id#73, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [2]: [customer_id#74, year_total#75] +Arguments: hashpartitioning(customer_id#74, 5), ENSURE_REQUIREMENTS, [id=#76] (77) Sort [codegen id : 34] -Input [2]: [customer_id#73, year_total#74] -Arguments: [customer_id#73 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#74, year_total#75] +Arguments: [customer_id#74 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#73] -Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#38) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#74] +Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#75) / promote_precision(year_total#57)), DecimalType(37,20), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#59) (79) Project [codegen id : 35] Output [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] -Input [9]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57, customer_id#73, year_total#74] +Input [8]: [customer_id#17, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#57, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#59, customer_id#74, year_total#75] (80) TakeOrderedAndProject Input [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] @@ -468,7 +468,7 @@ Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2001)) AND d_year#6 IN (2001, (84) BroadcastExchange Input [2]: [d_date_sk#5, d_year#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (88) @@ -493,10 +493,10 @@ Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (20 (88) BroadcastExchange Input [2]: [d_date_sk#24, d_year#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#23 +Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#23 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt index 99e72fe265b6b..e4c930b0c6f25 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (35) Project [customer_id,customer_first_name,customer_last_name] - SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END] InputAdapter WholeStageCodegen (26) - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] + Project [customer_id,customer_id,customer_first_name,customer_last_name,year_total,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) From f098c035bc35674710f1243d80c0e5f4934e39fc Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 1 Aug 2021 19:58:23 +0800 Subject: [PATCH 07/17] Fix --- .../spark/sql/execution/PlannerSuite.scala | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index fad6ed104fc80..5b44e26e438d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -994,17 +994,12 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { |WHERE t1id * 10 = t3.id3 * 10 """.stripMargin).queryExecution.executedPlan val sortNodes = collect(planned) { case s: SortExec => s } - assert(sortNodes.size == 3) + assert(sortNodes.size == 4) val exchangeNodes = collect(planned) { case e: ShuffleExchangeExec => e } - assert(exchangeNodes.size == 3) + assert(exchangeNodes.size == 4) val projects = collect(planned) { case p: ProjectExec => p } - assert(projects.exists(_.outputPartitioning match { - case HashPartitioning(Seq(Multiply(ar1: AttributeReference, _, _)), _) => - ar1.name == "t1id" - case _ => - false - })) + assert(!projects.exists(_.outputPartitioning.isInstanceOf[HashPartitioning])) } } } @@ -1103,13 +1098,13 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { // t12 is already sorted on `t1.id * 2`. and we need to sort it on `2 * t12.id` // for 2nd join. So sorting on t12 can be avoided val sortNodes = planned.collect { case s: SortExec => s } - assert(sortNodes.size == 3) + assert(sortNodes.size == 4) val outputOrdering = planned.outputOrdering assert(outputOrdering.size == 1) // Sort order should have 3 childrens, not 4. This is because t1.id*2 and 2*t1.id are same - assert(outputOrdering.head.children.size == 3) + assert(outputOrdering.head.children.size == 2) assert(outputOrdering.head.children.count(_.isInstanceOf[AttributeReference]) == 2) - assert(outputOrdering.head.children.count(_.isInstanceOf[Multiply]) == 1) + assert(outputOrdering.head.children.count(_.isInstanceOf[Multiply]) == 0) } } } From 6e65595ac65d05e6f612bee7d21c28d09c7ce7d0 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 9 Aug 2021 17:25:03 +0800 Subject: [PATCH 08/17] Move to Finish Analysis to infer more filters. --- .../sql/catalyst/optimizer/Optimizer.scala | 1 + .../optimizer/PullOutJoinCondition.scala | 3 +- .../spark/sql/execution/SparkOptimizer.scala | 8 +- .../approved-plans-v1_4/q11.sf100/explain.txt | 88 +-- .../q11.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q15.sf100/explain.txt | 100 +-- .../q15.sf100/simplified.txt | 4 +- .../q24a.sf100/explain.txt | 91 ++- .../q24a.sf100/simplified.txt | 15 +- .../q24b.sf100/explain.txt | 91 ++- .../q24b.sf100/simplified.txt | 15 +- .../approved-plans-v1_4/q4.sf100/explain.txt | 262 ++++---- .../q4.sf100/simplified.txt | 6 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 48 +- .../q47.sf100/simplified.txt | 12 +- .../approved-plans-v1_4/q57.sf100/explain.txt | 48 +- .../q57.sf100/simplified.txt | 12 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 567 +++++++++--------- .../q58.sf100/simplified.txt | 44 +- .../approved-plans-v1_4/q58/explain.txt | 545 ++++++++--------- .../approved-plans-v1_4/q58/simplified.txt | 60 +- .../approved-plans-v1_4/q8.sf100/explain.txt | 162 ++--- .../q8.sf100/simplified.txt | 12 +- .../approved-plans-v1_4/q83.sf100/explain.txt | 542 +++++++++-------- .../q83.sf100/simplified.txt | 50 +- .../approved-plans-v1_4/q83/explain.txt | 504 ++++++++-------- .../approved-plans-v1_4/q83/simplified.txt | 62 +- .../approved-plans-v2_7/q11.sf100/explain.txt | 88 +-- .../q11.sf100/simplified.txt | 4 +- .../approved-plans-v2_7/q47.sf100/explain.txt | 48 +- .../q47.sf100/simplified.txt | 12 +- .../approved-plans-v2_7/q57.sf100/explain.txt | 48 +- .../q57.sf100/simplified.txt | 12 +- .../approved-plans-v2_7/q74.sf100/explain.txt | 88 +-- .../q74.sf100/simplified.txt | 4 +- 35 files changed, 1855 insertions(+), 1805 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index bcf18b2140492..dcc6da9afd6c1 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 @@ -151,6 +151,7 @@ abstract class Optimizer(catalogManager: CatalogManager) ReplaceExpressions, RewriteNonCorrelatedExists, PullOutGroupingExpressions, + PullOutJoinCondition, ComputeCurrentTime, ReplaceCurrentLike(catalogManager)) :: ////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala index 2adeff0c1b150..b5384c653a657 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala @@ -44,8 +44,7 @@ object PullOutJoinCondition extends Rule[LogicalPlan] with JoinSelectionHelper with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(_.containsPattern(JOIN)) { - case j @ Join(left, right, _, Some(condition), _) - if j.resolved && !j.isStreaming && !canPlanAsBroadcastHashJoin(j, conf) => + case j @ Join(left, right, _, Some(condition), _) if j.resolved && !j.isStreaming => val complexExpressions = splitConjunctivePredicates(condition).flatMap(_.children).flatMap { case e: Expression if !e.foldable && e.children.nonEmpty => Seq(e) case _ => Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 01884006f10ee..dc3ceb5c595d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -42,16 +42,12 @@ class SparkOptimizer( override def defaultBatches: Seq[Batch] = (preOptimizationBatches ++ super.defaultBatches :+ Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+ Batch("PartitionPruning", Once, - PartitionPruning, - OptimizeSubqueries) :+ + PartitionPruning) :+ Batch("Pushdown Filters from PartitionPruning", fixedPoint, PushDownPredicates) :+ Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, - PruneFilters) :+ - Batch("Pull out join condition", Once, - PullOutJoinCondition, - CollapseProject)) ++ + PruneFilters)) ++ postHocOptimizationBatches :+ Batch("Extract Python UDFs", Once, ExtractPythonUDFFromJoinCondition, diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt index fcc62c8dea526..fa8839240f91e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt @@ -355,94 +355,94 @@ Right keys [1]: [customer_id#70] Join condition: None (61) Project [codegen id : 26] -Output [4]: [customer_id#22, customer_preferred_cust_flag#46, year_total#71, CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#73] +Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#71] Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, customer_id#70, year_total#71] (62) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77] +Output [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_sold_date_sk#77 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ws_sold_date_sk#76), dynamicpruningexpression(ws_sold_date_sk#76 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (63) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77] +Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] (64) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77] -Condition : isnotnull(ws_bill_customer_sk#74) +Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] +Condition : isnotnull(ws_bill_customer_sk#73) (65) ReusedExchange [Reuses operator id: 89] -Output [2]: [d_date_sk#78, d_year#79] +Output [2]: [d_date_sk#77, d_year#78] (66) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#77] -Right keys [1]: [d_date_sk#78] +Left keys [1]: [ws_sold_date_sk#76] +Right keys [1]: [d_date_sk#77] Join condition: None (67) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, ws_sold_date_sk#77, d_date_sk#78, d_year#79] +Output [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] +Input [6]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76, d_date_sk#77, d_year#78] (68) Exchange -Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] -Arguments: hashpartitioning(ws_bill_customer_sk#74, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] +Arguments: hashpartitioning(ws_bill_customer_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] (69) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] -Arguments: [ws_bill_customer_sk#74 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] +Arguments: [ws_bill_customer_sk#73 ASC NULLS FIRST], false, 0 (70) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#81, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88] +Output [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] (71) Sort [codegen id : 31] -Input [8]: [c_customer_sk#81, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88] -Arguments: [c_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] +Arguments: [c_customer_sk#80 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#74] -Right keys [1]: [c_customer_sk#81] +Left keys [1]: [ws_bill_customer_sk#73] +Right keys [1]: [c_customer_sk#80] Join condition: None (73) Project [codegen id : 32] -Output [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] -Input [12]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79, c_customer_sk#81, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88] +Output [10]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] +Input [12]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78, c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] (74) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, ws_ext_discount_amt#75, ws_ext_list_price#76, d_year#79] -Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#89] -Results [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, sum#90] +Input [10]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] +Keys [8]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#88] +Results [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] (75) Exchange -Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, sum#90] -Arguments: hashpartitioning(c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] +Arguments: hashpartitioning(c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#90] (76) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79, sum#90] -Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#86, c_login#87, c_email_address#88, d_year#79] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))#92] -Results [2]: [c_customer_id#82 AS customer_id#93, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#76 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(8,2)))), DecimalType(8,2), true)))#92,18,2) AS year_total#94] +Input [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] +Keys [8]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))#91] +Results [2]: [c_customer_id#81 AS customer_id#92, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))#91,18,2) AS year_total#93] (77) Exchange -Input [2]: [customer_id#93, year_total#94] -Arguments: hashpartitioning(customer_id#93, 5), ENSURE_REQUIREMENTS, [id=#95] +Input [2]: [customer_id#92, year_total#93] +Arguments: hashpartitioning(customer_id#92, 5), ENSURE_REQUIREMENTS, [id=#94] (78) Sort [codegen id : 34] -Input [2]: [customer_id#93, year_total#94] -Arguments: [customer_id#93 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#92, year_total#93] +Arguments: [customer_id#92 ASC NULLS FIRST], false, 0 (79) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#93] -Join condition: (CASE WHEN (year_total#71 > 0.00) THEN CheckOverflow((promote_precision(year_total#94) / promote_precision(year_total#71)), DecimalType(38,20), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#73) +Right keys [1]: [customer_id#92] +Join condition: (CASE WHEN (year_total#71 > 0.00) THEN CheckOverflow((promote_precision(year_total#93) / promote_precision(year_total#71)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END) (80) Project [codegen id : 35] Output [1]: [customer_preferred_cust_flag#46] -Input [6]: [customer_id#22, customer_preferred_cust_flag#46, year_total#71, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END#73, customer_id#93, year_total#94] +Input [7]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#71, customer_id#92, year_total#93] (81) TakeOrderedAndProject Input [1]: [customer_preferred_cust_flag#46] @@ -473,7 +473,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2001)) AND isnotnull(d_date_sk (85) BroadcastExchange Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (89) @@ -498,10 +498,10 @@ Condition : ((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND isnotnull(d_date_ (89) BroadcastExchange Input [2]: [d_date_sk#30, d_year#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#29 +Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#76 IN dynamicpruning#29 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt index 7162fb28da5ce..889fc666bd810 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (35) Project [customer_preferred_cust_flag] - SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,20)) END] + SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (26) - Project [customer_id,customer_preferred_cust_flag,year_total,year_total,year_total] + Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt index 6be8325f6af7b..55c39ed2e3288 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt @@ -53,108 +53,108 @@ Right keys [1]: [d_date_sk#5] Join condition: None (6) Project [codegen id : 2] -Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, (cs_sales_price#2 > 500.00) AS (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6] +Output [2]: [cs_bill_customer_sk#1, cs_sales_price#2] Input [4]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, d_date_sk#5] (7) Exchange -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#7] +Input [2]: [cs_bill_customer_sk#1, cs_sales_price#2] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#6] (8) Sort [codegen id : 3] -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6] +Input [2]: [cs_bill_customer_sk#1, cs_sales_price#2] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (9) Scan parquet default.customer -Output [2]: [c_customer_sk#8, c_current_addr_sk#9] +Output [2]: [c_customer_sk#7, c_current_addr_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 4] -Input [2]: [c_customer_sk#8, c_current_addr_sk#9] +Input [2]: [c_customer_sk#7, c_current_addr_sk#8] (11) Filter [codegen id : 4] -Input [2]: [c_customer_sk#8, c_current_addr_sk#9] -Condition : (isnotnull(c_customer_sk#8) AND isnotnull(c_current_addr_sk#9)) +Input [2]: [c_customer_sk#7, c_current_addr_sk#8] +Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) (12) Exchange -Input [2]: [c_customer_sk#8, c_current_addr_sk#9] -Arguments: hashpartitioning(c_current_addr_sk#9, 5), ENSURE_REQUIREMENTS, [id=#10] +Input [2]: [c_customer_sk#7, c_current_addr_sk#8] +Arguments: hashpartitioning(c_current_addr_sk#8, 5), ENSURE_REQUIREMENTS, [id=#9] (13) Sort [codegen id : 5] -Input [2]: [c_customer_sk#8, c_current_addr_sk#9] -Arguments: [c_current_addr_sk#9 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#7, c_current_addr_sk#8] +Arguments: [c_current_addr_sk#8 ASC NULLS FIRST], false, 0 (14) Scan parquet default.customer_address -Output [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] +Output [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 6] -Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] +Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] (16) Filter [codegen id : 6] -Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] -Condition : isnotnull(ca_address_sk#11) +Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] +Condition : isnotnull(ca_address_sk#10) (17) Exchange -Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] -Arguments: hashpartitioning(ca_address_sk#11, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] +Arguments: hashpartitioning(ca_address_sk#10, 5), ENSURE_REQUIREMENTS, [id=#13] (18) Sort [codegen id : 7] -Input [3]: [ca_address_sk#11, ca_state#12, ca_zip#13] -Arguments: [ca_address_sk#11 ASC NULLS FIRST], false, 0 +Input [3]: [ca_address_sk#10, ca_state#11, ca_zip#12] +Arguments: [ca_address_sk#10 ASC NULLS FIRST], false, 0 (19) SortMergeJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#9] -Right keys [1]: [ca_address_sk#11] +Left keys [1]: [c_current_addr_sk#8] +Right keys [1]: [ca_address_sk#10] Join condition: None (20) Project [codegen id : 8] -Output [3]: [c_customer_sk#8, ca_zip#13, (substr(ca_zip#13, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#12 IN (CA,WA,GA)) AS ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] -Input [5]: [c_customer_sk#8, c_current_addr_sk#9, ca_address_sk#11, ca_state#12, ca_zip#13] +Output [3]: [c_customer_sk#7, ca_state#11, ca_zip#12] +Input [5]: [c_customer_sk#7, c_current_addr_sk#8, ca_address_sk#10, ca_state#11, ca_zip#12] (21) Exchange -Input [3]: [c_customer_sk#8, ca_zip#13, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] -Arguments: hashpartitioning(c_customer_sk#8, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [3]: [c_customer_sk#7, ca_state#11, ca_zip#12] +Arguments: hashpartitioning(c_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] (22) Sort [codegen id : 9] -Input [3]: [c_customer_sk#8, ca_zip#13, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] -Arguments: [c_customer_sk#8 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#7, ca_state#11, ca_zip#12] +Arguments: [c_customer_sk#7 ASC NULLS FIRST], false, 0 (23) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#8] -Join condition: (((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15 OR (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6) +Right keys [1]: [c_customer_sk#7] +Join condition: ((substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#11 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) (24) Project [codegen id : 10] -Output [2]: [cs_sales_price#2, ca_zip#13] -Input [6]: [cs_bill_customer_sk#1, cs_sales_price#2, (spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)#6, c_customer_sk#8, ca_zip#13, ((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA')))#15] +Output [2]: [cs_sales_price#2, ca_zip#12] +Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, c_customer_sk#7, ca_state#11, ca_zip#12] (25) HashAggregate [codegen id : 10] -Input [2]: [cs_sales_price#2, ca_zip#13] -Keys [1]: [ca_zip#13] +Input [2]: [cs_sales_price#2, ca_zip#12] +Keys [1]: [ca_zip#12] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum#17] -Results [2]: [ca_zip#13, sum#18] +Aggregate Attributes [1]: [sum#15] +Results [2]: [ca_zip#12, sum#16] (26) Exchange -Input [2]: [ca_zip#13, sum#18] -Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, [id=#19] +Input [2]: [ca_zip#12, sum#16] +Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [id=#17] (27) HashAggregate [codegen id : 11] -Input [2]: [ca_zip#13, sum#18] -Keys [1]: [ca_zip#13] +Input [2]: [ca_zip#12, sum#16] +Keys [1]: [ca_zip#12] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#20] -Results [2]: [ca_zip#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#20,17,2) AS sum(cs_sales_price)#21] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#18] +Results [2]: [ca_zip#12, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#18,17,2) AS sum(cs_sales_price)#19] (28) TakeOrderedAndProject -Input [2]: [ca_zip#13, sum(cs_sales_price)#21] -Arguments: 100, [ca_zip#13 ASC NULLS FIRST], [ca_zip#13, sum(cs_sales_price)#21] +Input [2]: [ca_zip#12, sum(cs_sales_price)#19] +Arguments: 100, [ca_zip#12 ASC NULLS FIRST], [ca_zip#12, sum(cs_sales_price)#19] ===== Subqueries ===== @@ -167,25 +167,25 @@ BroadcastExchange (33) (29) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#22, d_qoy#23] +Output [3]: [d_date_sk#5, d_year#20, d_qoy#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] +Input [3]: [d_date_sk#5, d_year#20, d_qoy#21] (31) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] -Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 2)) AND (d_year#22 = 2001)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#5, d_year#20, d_qoy#21] +Condition : ((((isnotnull(d_qoy#21) AND isnotnull(d_year#20)) AND (d_qoy#21 = 2)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#5)) (32) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] +Input [3]: [d_date_sk#5, d_year#20, d_qoy#21] (33) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt index fee4d9431c8f2..8105f67555c09 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (10) HashAggregate [ca_zip,cs_sales_price] [sum,sum] Project [cs_sales_price,ca_zip] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk,((substr(spark_catalog.default.customer_address.ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792')) OR (spark_catalog.default.customer_address.ca_state IN ('CA', 'WA', 'GA'))),(spark_catalog.default.catalog_sales.cs_sales_price > 500.00BD)] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk,ca_zip,ca_state,cs_sales_price] InputAdapter WholeStageCodegen (3) Sort [cs_bill_customer_sk] @@ -35,7 +35,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (8) - Project [c_customer_sk,ca_zip,ca_state] + Project [c_customer_sk,ca_state,ca_zip] SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (5) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt index a85878eeb26ed..b57eca2578e00 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt @@ -278,14 +278,14 @@ Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subqu ===== Subqueries ===== Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (97) -+- Exchange (96) - +- * HashAggregate (95) - +- * HashAggregate (94) - +- Exchange (93) - +- * HashAggregate (92) - +- * Project (91) - +- * SortMergeJoin Inner (90) +* HashAggregate (96) ++- Exchange (95) + +- * HashAggregate (94) + +- * HashAggregate (93) + +- Exchange (92) + +- * HashAggregate (91) + +- * Project (90) + +- * SortMergeJoin Inner (89) :- * Sort (83) : +- Exchange (82) : +- * Project (81) @@ -320,12 +320,11 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer : : +- ReusedExchange (72) : +- * Sort (79) : +- ReusedExchange (78) - +- * Sort (89) - +- Exchange (88) - +- * Project (87) - +- * Filter (86) - +- * ColumnarToRow (85) - +- Scan parquet default.customer_address (84) + +- * Sort (88) + +- Exchange (87) + +- * Filter (86) + +- * ColumnarToRow (85) + +- Scan parquet default.customer_address (84) (50) Scan parquet default.store_sales @@ -484,63 +483,59 @@ Input [3]: [ca_state#78, ca_zip#79, ca_country#80] (86) Filter [codegen id : 16] Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Condition : ((isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) AND isnotnull(upper(ca_country#80))) +Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) -(87) Project [codegen id : 16] -Output [3]: [ca_state#78, ca_zip#79, upper(ca_country#80) AS upper(spark_catalog.default.customer_address.ca_country)#81] +(87) Exchange Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Arguments: hashpartitioning(upper(ca_country#80), ca_zip#79, 5), ENSURE_REQUIREMENTS, [id=#81] -(88) Exchange -Input [3]: [ca_state#78, ca_zip#79, upper(spark_catalog.default.customer_address.ca_country)#81] -Arguments: hashpartitioning(upper(spark_catalog.default.customer_address.ca_country)#81, ca_zip#79, 5), ENSURE_REQUIREMENTS, [id=#82] - -(89) Sort [codegen id : 17] -Input [3]: [ca_state#78, ca_zip#79, upper(spark_catalog.default.customer_address.ca_country)#81] -Arguments: [upper(spark_catalog.default.customer_address.ca_country)#81 ASC NULLS FIRST, ca_zip#79 ASC NULLS FIRST], false, 0 +(88) Sort [codegen id : 17] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Arguments: [upper(ca_country#80) ASC NULLS FIRST, ca_zip#79 ASC NULLS FIRST], false, 0 -(90) SortMergeJoin [codegen id : 18] +(89) SortMergeJoin [codegen id : 18] Left keys [2]: [c_birth_country#73, s_zip#59] -Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#81, ca_zip#79] +Right keys [2]: [upper(ca_country#80), ca_zip#79] Join condition: None -(91) Project [codegen id : 18] +(90) Project [codegen id : 18] Output [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#78] -Input [15]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, ca_state#78, ca_zip#79, upper(spark_catalog.default.customer_address.ca_country)#81] +Input [15]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, ca_state#78, ca_zip#79, ca_country#80] -(92) HashAggregate [codegen id : 18] +(91) HashAggregate [codegen id : 18] Input [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#78] Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#83] -Results [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#84] +Aggregate Attributes [1]: [sum#82] +Results [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] -(93) Exchange -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#84] -Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#85] +(92) Exchange +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] +Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#84] -(94) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#84] +(93) HashAggregate [codegen id : 19] +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#86] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#86,17,2) AS netpaid#39] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#39] -(95) HashAggregate [codegen id : 19] +(94) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#87, count#88] -Results [2]: [sum#89, count#90] +Aggregate Attributes [2]: [sum#86, count#87] +Results [2]: [sum#88, count#89] -(96) Exchange -Input [2]: [sum#89, count#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] +(95) Exchange +Input [2]: [sum#88, count#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] -(97) HashAggregate [codegen id : 20] -Input [2]: [sum#89, count#90] +(96) HashAggregate [codegen id : 20] +Input [2]: [sum#88, count#89] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#92] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#92)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#93] +Aggregate Attributes [1]: [avg(netpaid#39)#91] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt index bb65ace10c8e9..11f1b6b71dc01 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt @@ -13,7 +13,7 @@ WholeStageCodegen (14) WholeStageCodegen (18) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - SortMergeJoin [c_birth_country,s_zip,upper(spark_catalog.default.customer_address.ca_country),ca_zip] + SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (15) Sort [c_birth_country,s_zip] @@ -81,15 +81,14 @@ WholeStageCodegen (14) ReusedExchange [sr_item_sk,sr_ticket_number] #7 InputAdapter WholeStageCodegen (17) - Sort [upper(spark_catalog.default.customer_address.ca_country),ca_zip] + Sort [ca_country,ca_zip] InputAdapter - Exchange [upper(spark_catalog.default.customer_address.ca_country),ca_zip] #18 + Exchange [ca_country,ca_zip] #18 WholeStageCodegen (16) - Project [ca_state,ca_zip,ca_country] - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + Filter [ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name,s_store_name] #1 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt index 5e87b8e585341..1814c7de8204d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt @@ -278,14 +278,14 @@ Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subqu ===== Subqueries ===== Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (97) -+- Exchange (96) - +- * HashAggregate (95) - +- * HashAggregate (94) - +- Exchange (93) - +- * HashAggregate (92) - +- * Project (91) - +- * SortMergeJoin Inner (90) +* HashAggregate (96) ++- Exchange (95) + +- * HashAggregate (94) + +- * HashAggregate (93) + +- Exchange (92) + +- * HashAggregate (91) + +- * Project (90) + +- * SortMergeJoin Inner (89) :- * Sort (83) : +- Exchange (82) : +- * Project (81) @@ -320,12 +320,11 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer : : +- ReusedExchange (72) : +- * Sort (79) : +- ReusedExchange (78) - +- * Sort (89) - +- Exchange (88) - +- * Project (87) - +- * Filter (86) - +- * ColumnarToRow (85) - +- Scan parquet default.customer_address (84) + +- * Sort (88) + +- Exchange (87) + +- * Filter (86) + +- * ColumnarToRow (85) + +- Scan parquet default.customer_address (84) (50) Scan parquet default.store_sales @@ -484,63 +483,59 @@ Input [3]: [ca_state#78, ca_zip#79, ca_country#80] (86) Filter [codegen id : 16] Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Condition : ((isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) AND isnotnull(upper(ca_country#80))) +Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) -(87) Project [codegen id : 16] -Output [3]: [ca_state#78, ca_zip#79, upper(ca_country#80) AS upper(spark_catalog.default.customer_address.ca_country)#81] +(87) Exchange Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Arguments: hashpartitioning(upper(ca_country#80), ca_zip#79, 5), ENSURE_REQUIREMENTS, [id=#81] -(88) Exchange -Input [3]: [ca_state#78, ca_zip#79, upper(spark_catalog.default.customer_address.ca_country)#81] -Arguments: hashpartitioning(upper(spark_catalog.default.customer_address.ca_country)#81, ca_zip#79, 5), ENSURE_REQUIREMENTS, [id=#82] - -(89) Sort [codegen id : 17] -Input [3]: [ca_state#78, ca_zip#79, upper(spark_catalog.default.customer_address.ca_country)#81] -Arguments: [upper(spark_catalog.default.customer_address.ca_country)#81 ASC NULLS FIRST, ca_zip#79 ASC NULLS FIRST], false, 0 +(88) Sort [codegen id : 17] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Arguments: [upper(ca_country#80) ASC NULLS FIRST, ca_zip#79 ASC NULLS FIRST], false, 0 -(90) SortMergeJoin [codegen id : 18] +(89) SortMergeJoin [codegen id : 18] Left keys [2]: [c_birth_country#73, s_zip#59] -Right keys [2]: [upper(spark_catalog.default.customer_address.ca_country)#81, ca_zip#79] +Right keys [2]: [upper(ca_country#80), ca_zip#79] Join condition: None -(91) Project [codegen id : 18] +(90) Project [codegen id : 18] Output [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#78] -Input [15]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, ca_state#78, ca_zip#79, upper(spark_catalog.default.customer_address.ca_country)#81] +Input [15]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, ca_state#78, ca_zip#79, ca_country#80] -(92) HashAggregate [codegen id : 18] +(91) HashAggregate [codegen id : 18] Input [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#78] Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#83] -Results [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#84] +Aggregate Attributes [1]: [sum#82] +Results [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] -(93) Exchange -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#84] -Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#85] +(92) Exchange +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] +Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#84] -(94) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#84] +(93) HashAggregate [codegen id : 19] +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#86] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#86,17,2) AS netpaid#39] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#39] -(95) HashAggregate [codegen id : 19] +(94) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#87, count#88] -Results [2]: [sum#89, count#90] +Aggregate Attributes [2]: [sum#86, count#87] +Results [2]: [sum#88, count#89] -(96) Exchange -Input [2]: [sum#89, count#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] +(95) Exchange +Input [2]: [sum#88, count#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] -(97) HashAggregate [codegen id : 20] -Input [2]: [sum#89, count#90] +(96) HashAggregate [codegen id : 20] +Input [2]: [sum#88, count#89] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#92] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#92)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#93] +Aggregate Attributes [1]: [avg(netpaid#39)#91] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt index bb65ace10c8e9..11f1b6b71dc01 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt @@ -13,7 +13,7 @@ WholeStageCodegen (14) WholeStageCodegen (18) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - SortMergeJoin [c_birth_country,s_zip,upper(spark_catalog.default.customer_address.ca_country),ca_zip] + SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (15) Sort [c_birth_country,s_zip] @@ -81,15 +81,14 @@ WholeStageCodegen (14) ReusedExchange [sr_item_sk,sr_ticket_number] #7 InputAdapter WholeStageCodegen (17) - Sort [upper(spark_catalog.default.customer_address.ca_country),ca_zip] + Sort [ca_country,ca_zip] InputAdapter - Exchange [upper(spark_catalog.default.customer_address.ca_country),ca_zip] #18 + Exchange [ca_country,ca_zip] #18 WholeStageCodegen (16) - Project [ca_state,ca_zip,ca_country] - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + Filter [ca_country,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name,s_store_name] #1 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt index 107d5fb85a415..4e4dd9555c718 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt @@ -390,274 +390,274 @@ Right keys [1]: [customer_id#87] Join condition: None (60) Project [codegen id : 26] -Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#60) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#90] +Output [11]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#88] Input [12]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, customer_id#87, year_total#88] (61) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96] +Output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#96), dynamicpruningexpression(cs_sold_date_sk#96 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(cs_sold_date_sk#95), dynamicpruningexpression(cs_sold_date_sk#95 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 28] -Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96] +Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] (63) Filter [codegen id : 28] -Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96] -Condition : isnotnull(cs_bill_customer_sk#91) +Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] +Condition : isnotnull(cs_bill_customer_sk#90) (64) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#97, d_year#98] +Output [2]: [d_date_sk#96, d_year#97] (65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#96] -Right keys [1]: [d_date_sk#97] +Left keys [1]: [cs_sold_date_sk#95] +Right keys [1]: [d_date_sk#96] Join condition: None (66) Project [codegen id : 28] -Output [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] -Input [8]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, cs_sold_date_sk#96, d_date_sk#97, d_year#98] +Output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] +Input [8]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95, d_date_sk#96, d_year#97] (67) Exchange -Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] -Arguments: hashpartitioning(cs_bill_customer_sk#91, 5), ENSURE_REQUIREMENTS, [id=#99] +Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] +Arguments: hashpartitioning(cs_bill_customer_sk#90, 5), ENSURE_REQUIREMENTS, [id=#98] (68) Sort [codegen id : 29] -Input [6]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] -Arguments: [cs_bill_customer_sk#91 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] +Arguments: [cs_bill_customer_sk#90 ASC NULLS FIRST], false, 0 (69) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#100, c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107] +Output [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] (70) Sort [codegen id : 31] -Input [8]: [c_customer_sk#100, c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107] -Arguments: [c_customer_sk#100 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] +Arguments: [c_customer_sk#99 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 32] -Left keys [1]: [cs_bill_customer_sk#91] -Right keys [1]: [c_customer_sk#100] +Left keys [1]: [cs_bill_customer_sk#90] +Right keys [1]: [c_customer_sk#99] Join condition: None (72) Project [codegen id : 32] -Output [12]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] -Input [14]: [cs_bill_customer_sk#91, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98, c_customer_sk#100, c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107] +Output [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] +Input [14]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97, c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] (73) HashAggregate [codegen id : 32] -Input [12]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, cs_ext_discount_amt#92, cs_ext_sales_price#93, cs_ext_wholesale_cost#94, cs_ext_list_price#95, d_year#98] -Keys [8]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#108, isEmpty#109] -Results [10]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, sum#110, isEmpty#111] +Input [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#97] +Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#107, isEmpty#108] +Results [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, sum#109, isEmpty#110] (74) Exchange -Input [10]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, sum#110, isEmpty#111] -Arguments: hashpartitioning(c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, 5), ENSURE_REQUIREMENTS, [id=#112] +Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, sum#109, isEmpty#110] +Arguments: hashpartitioning(c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, 5), ENSURE_REQUIREMENTS, [id=#111] (75) HashAggregate [codegen id : 33] -Input [10]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98, sum#110, isEmpty#111] -Keys [8]: [c_customer_id#101, c_first_name#102, c_last_name#103, c_preferred_cust_flag#104, c_birth_country#105, c_login#106, c_email_address#107, d_year#98] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#113] -Results [2]: [c_customer_id#101 AS customer_id#114, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#95 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#94 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#92 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#93 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#113 AS year_total#115] +Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97, sum#109, isEmpty#110] +Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#97] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112] +Results [2]: [c_customer_id#100 AS customer_id#113, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#94 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#93 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#91 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#92 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112 AS year_total#114] (76) Exchange -Input [2]: [customer_id#114, year_total#115] -Arguments: hashpartitioning(customer_id#114, 5), ENSURE_REQUIREMENTS, [id=#116] +Input [2]: [customer_id#113, year_total#114] +Arguments: hashpartitioning(customer_id#113, 5), ENSURE_REQUIREMENTS, [id=#115] (77) Sort [codegen id : 34] -Input [2]: [customer_id#114, year_total#115] -Arguments: [customer_id#114 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#113, year_total#114] +Arguments: [customer_id#113 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#114] -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#115) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#90) +Right keys [1]: [customer_id#113] +Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#60) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END) (79) Project [codegen id : 35] -Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#115] -Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#90, customer_id#114, year_total#115] +Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114] +Input [13]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#88, customer_id#113, year_total#114] (80) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] +Output [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#122), dynamicpruningexpression(ws_sold_date_sk#122 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 37] -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] +Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121] (82) Filter [codegen id : 37] -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Condition : isnotnull(ws_bill_customer_sk#117) +Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121] +Condition : isnotnull(ws_bill_customer_sk#116) (83) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#123, d_year#124] +Output [2]: [d_date_sk#122, d_year#123] (84) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_sold_date_sk#122] -Right keys [1]: [d_date_sk#123] +Left keys [1]: [ws_sold_date_sk#121] +Right keys [1]: [d_date_sk#122] Join condition: None (85) Project [codegen id : 37] -Output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] -Input [8]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122, d_date_sk#123, d_year#124] +Output [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] +Input [8]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, ws_sold_date_sk#121, d_date_sk#122, d_year#123] (86) Exchange -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] -Arguments: hashpartitioning(ws_bill_customer_sk#117, 5), ENSURE_REQUIREMENTS, [id=#125] +Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] +Arguments: hashpartitioning(ws_bill_customer_sk#116, 5), ENSURE_REQUIREMENTS, [id=#124] (87) Sort [codegen id : 38] -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] -Arguments: [ws_bill_customer_sk#117 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] +Arguments: [ws_bill_customer_sk#116 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#126, c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133] +Output [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] (89) Sort [codegen id : 40] -Input [8]: [c_customer_sk#126, c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133] -Arguments: [c_customer_sk#126 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +Arguments: [c_customer_sk#125 ASC NULLS FIRST], false, 0 (90) SortMergeJoin [codegen id : 41] -Left keys [1]: [ws_bill_customer_sk#117] -Right keys [1]: [c_customer_sk#126] +Left keys [1]: [ws_bill_customer_sk#116] +Right keys [1]: [c_customer_sk#125] Join condition: None (91) Project [codegen id : 41] -Output [12]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] -Input [14]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124, c_customer_sk#126, c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133] +Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] +Input [14]: [ws_bill_customer_sk#116, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123, c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] (92) HashAggregate [codegen id : 41] -Input [12]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#124] -Keys [8]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#134, isEmpty#135] -Results [10]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, sum#136, isEmpty#137] +Input [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#117, ws_ext_sales_price#118, ws_ext_wholesale_cost#119, ws_ext_list_price#120, d_year#123] +Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#133, isEmpty#134] +Results [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, sum#135, isEmpty#136] (93) Exchange -Input [10]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, sum#136, isEmpty#137] -Arguments: hashpartitioning(c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, 5), ENSURE_REQUIREMENTS, [id=#138] +Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, sum#135, isEmpty#136] +Arguments: hashpartitioning(c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, 5), ENSURE_REQUIREMENTS, [id=#137] (94) HashAggregate [codegen id : 42] -Input [10]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124, sum#136, isEmpty#137] -Keys [8]: [c_customer_id#127, c_first_name#128, c_last_name#129, c_preferred_cust_flag#130, c_birth_country#131, c_login#132, c_email_address#133, d_year#124] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#139] -Results [2]: [c_customer_id#127 AS customer_id#140, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#121 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#120 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#118 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#119 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#139 AS year_total#141] +Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123, sum#135, isEmpty#136] +Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#123] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138] +Results [2]: [c_customer_id#126 AS customer_id#139, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#120 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#119 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#117 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#118 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138 AS year_total#140] (95) Filter [codegen id : 42] -Input [2]: [customer_id#140, year_total#141] -Condition : (isnotnull(year_total#141) AND (year_total#141 > 0.000000)) +Input [2]: [customer_id#139, year_total#140] +Condition : (isnotnull(year_total#140) AND (year_total#140 > 0.000000)) (96) Project [codegen id : 42] -Output [2]: [customer_id#140 AS customer_id#142, year_total#141 AS year_total#143] -Input [2]: [customer_id#140, year_total#141] +Output [2]: [customer_id#139 AS customer_id#141, year_total#140 AS year_total#142] +Input [2]: [customer_id#139, year_total#140] (97) Exchange -Input [2]: [customer_id#142, year_total#143] -Arguments: hashpartitioning(customer_id#142, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [2]: [customer_id#141, year_total#142] +Arguments: hashpartitioning(customer_id#141, 5), ENSURE_REQUIREMENTS, [id=#143] (98) Sort [codegen id : 43] -Input [2]: [customer_id#142, year_total#143] -Arguments: [customer_id#142 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#141, year_total#142] +Arguments: [customer_id#141 ASC NULLS FIRST], false, 0 (99) SortMergeJoin [codegen id : 44] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#142] +Right keys [1]: [customer_id#141] Join condition: None (100) Project [codegen id : 44] -Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#143, CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#115) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END AS CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#145] -Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#115, customer_id#142, year_total#143] +Output [11]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114, year_total#142] +Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114, customer_id#141, year_total#142] (101) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] +Output [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#151), dynamicpruningexpression(ws_sold_date_sk#151 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(ws_sold_date_sk#149), dynamicpruningexpression(ws_sold_date_sk#149 IN dynamicpruning#35)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 46] -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] +Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] (103) Filter [codegen id : 46] -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] -Condition : isnotnull(ws_bill_customer_sk#146) +Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] +Condition : isnotnull(ws_bill_customer_sk#144) (104) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#150, d_year#151] (105) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [ws_sold_date_sk#151] -Right keys [1]: [d_date_sk#152] +Left keys [1]: [ws_sold_date_sk#149] +Right keys [1]: [d_date_sk#150] Join condition: None (106) Project [codegen id : 46] -Output [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Input [8]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151, d_date_sk#152, d_year#153] +Output [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] +Input [8]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149, d_date_sk#150, d_year#151] (107) Exchange -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Arguments: hashpartitioning(ws_bill_customer_sk#146, 5), ENSURE_REQUIREMENTS, [id=#154] +Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] +Arguments: hashpartitioning(ws_bill_customer_sk#144, 5), ENSURE_REQUIREMENTS, [id=#152] (108) Sort [codegen id : 47] -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Arguments: [ws_bill_customer_sk#146 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] +Arguments: [ws_bill_customer_sk#144 ASC NULLS FIRST], false, 0 (109) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] +Output [8]: [c_customer_sk#153, c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160] (110) Sort [codegen id : 49] -Input [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] -Arguments: [c_customer_sk#155 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#153, c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160] +Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 (111) SortMergeJoin [codegen id : 50] -Left keys [1]: [ws_bill_customer_sk#146] -Right keys [1]: [c_customer_sk#155] +Left keys [1]: [ws_bill_customer_sk#144] +Right keys [1]: [c_customer_sk#153] Join condition: None (112) Project [codegen id : 50] -Output [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Input [14]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153, c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] +Output [12]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] +Input [14]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151, c_customer_sk#153, c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160] (113) HashAggregate [codegen id : 50] -Input [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#163, isEmpty#164] -Results [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Input [12]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#151] +Keys [8]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#161, isEmpty#162] +Results [10]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, sum#163, isEmpty#164] (114) Exchange -Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] -Arguments: hashpartitioning(c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [10]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, sum#163, isEmpty#164] +Arguments: hashpartitioning(c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, 5), ENSURE_REQUIREMENTS, [id=#165] (115) HashAggregate [codegen id : 51] -Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] -Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#168] -Results [2]: [c_customer_id#156 AS customer_id#169, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#168 AS year_total#170] +Input [10]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151, sum#163, isEmpty#164] +Keys [8]: [c_customer_id#154, c_first_name#155, c_last_name#156, c_preferred_cust_flag#157, c_birth_country#158, c_login#159, c_email_address#160, d_year#151] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166] +Results [2]: [c_customer_id#154 AS customer_id#167, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166 AS year_total#168] (116) Exchange -Input [2]: [customer_id#169, year_total#170] -Arguments: hashpartitioning(customer_id#169, 5), ENSURE_REQUIREMENTS, [id=#171] +Input [2]: [customer_id#167, year_total#168] +Arguments: hashpartitioning(customer_id#167, 5), ENSURE_REQUIREMENTS, [id=#169] (117) Sort [codegen id : 52] -Input [2]: [customer_id#169, year_total#170] -Arguments: [customer_id#169 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#167, year_total#168] +Arguments: [customer_id#167 ASC NULLS FIRST], false, 0 (118) SortMergeJoin [codegen id : 53] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#169] -Join condition: (CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#145 > CASE WHEN (year_total#143 > 0.000000) THEN CheckOverflow((promote_precision(year_total#170) / promote_precision(year_total#143)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#167] +Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#142 > 0.000000) THEN CheckOverflow((promote_precision(year_total#168) / promote_precision(year_total#142)), DecimalType(38,14), true) ELSE null END) (119) Project [codegen id : 53] Output [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] -Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#143, CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END#145, customer_id#169, year_total#170] +Input [13]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#88, year_total#114, year_total#142, customer_id#167, year_total#168] (120) TakeOrderedAndProject Input [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] @@ -688,7 +688,7 @@ Condition : ((isnotnull(d_year#9) AND (d_year#9 = 2001)) AND isnotnull(d_date_sk (124) BroadcastExchange Input [2]: [d_date_sk#8, d_year#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#172] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#170] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#34 IN dynamicpruning#35 BroadcastExchange (128) @@ -713,14 +713,14 @@ Condition : ((isnotnull(d_year#37) AND (d_year#37 = 2002)) AND isnotnull(d_date_ (128) BroadcastExchange Input [2]: [d_date_sk#36, d_year#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#173] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#171] Subquery:3 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#7 -Subquery:4 Hosting operator id = 61 Hosting Expression = cs_sold_date_sk#96 IN dynamicpruning#35 +Subquery:4 Hosting operator id = 61 Hosting Expression = cs_sold_date_sk#95 IN dynamicpruning#35 -Subquery:5 Hosting operator id = 80 Hosting Expression = ws_sold_date_sk#122 IN dynamicpruning#7 +Subquery:5 Hosting operator id = 80 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#7 -Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#151 IN dynamicpruning#35 +Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#149 IN dynamicpruning#35 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt index eab288e1d9891..d7e0a660bab5b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] WholeStageCodegen (53) Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - SortMergeJoin [customer_id,customer_id,CASE WHEN (t_c_firstyear.year_total > 0.000000BD) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END,year_total,year_total] + SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (44) Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] @@ -9,10 +9,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter WholeStageCodegen (35) Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.000000BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(38,14)) END] + SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (26) - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) 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 a612378242f1e..686036a835e25 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 @@ -223,61 +223,61 @@ Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_yea Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (rn#39 + 1) AS (v1_lag.rn + 1)#40] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] (40) Exchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] (41) Sort [codegen id : 23] -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (v1_lag.rn + 1)#40 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None (43) Project [codegen id : 24] Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37] -Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] +Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] (44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Output [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] (45) Sort [codegen id : 33] -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST], false, 0 (46) Window -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] +Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Arguments: [rank(d_year#45, d_moy#46) windowspecdefinition(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#48], [i_category#41, i_brand#42, s_store_name#43, s_company_name#44], [d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (rn#49 - 1) AS (v1_lead.rn - 1)#50] -Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] +Output [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Input [8]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47, rn#48] (48) Exchange -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] -Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Arguments: hashpartitioning(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1), 5), ENSURE_REQUIREMENTS, [id=#49] (49) Sort [codegen id : 35] -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (v1_lead.rn - 1)#50 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, (rn#48 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50] +Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None (51) Project [codegen id : 36] -Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#52, sum_sales#48 AS nsum#53] -Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] +Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] +Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] (52) TakeOrderedAndProject -Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] +Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt index 5c9d3bb5049c0..07c75d91ca3cf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] WholeStageCodegen (36) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,s_store_name,s_company_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] #8 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #8 WholeStageCodegen (22) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] #10 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #10 WholeStageCodegen (34) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter 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 b7ba9e2c06424..0f22f27fed99b 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 @@ -223,61 +223,61 @@ Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sale Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (rn#37 + 1) AS (v1_lag.rn + 1)#38] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] (40) Exchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#38] (41) Sort [codegen id : 23] -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (v1_lag.rn + 1)#38 ASC NULLS FIRST], false, 0 +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None (43) Project [codegen id : 24] Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] (44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] (45) Sort [codegen id : 33] -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 (46) Window -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] +Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#39, i_brand#40, cc_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#45], [i_category#39, i_brand#40, cc_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (rn#46 - 1) AS (v1_lead.rn - 1)#47] -Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] +Output [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] (48) Exchange -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] -Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: hashpartitioning(i_category#39, i_brand#40, cc_name#41, (rn#45 - 1), 5), ENSURE_REQUIREMENTS, [id=#46] (49) Sort [codegen id : 35] -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (v1_lead.rn - 1)#47 ASC NULLS FIRST], false, 0 +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, (rn#45 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47] +Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None (51) Project [codegen id : 36] -Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#49, sum_sales#45 AS nsum#50] -Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] +Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] +Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] (52) TakeOrderedAndProject -Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] +Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt index f3ef99b986f71..3bf10f82e6a88 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] WholeStageCodegen (36) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lead.rn - 1)] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lag.rn + 1)] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,cc_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,cc_name,(v1_lag.rn + 1)] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [i_category,i_brand,cc_name,(v1_lag.rn + 1)] #8 + Exchange [i_category,i_brand,cc_name,rn] #8 WholeStageCodegen (22) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,cc_name,(v1_lead.rn - 1)] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [i_category,i_brand,cc_name,(v1_lead.rn - 1)] #10 + Exchange [i_category,i_brand,cc_name,rn] #10 WholeStageCodegen (34) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index d6b61f63b41a8..8e969096c5239 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -1,70 +1,60 @@ == Physical Plan == -TakeOrderedAndProject (59) -+- * Project (58) - +- * BroadcastHashJoin Inner BuildRight (57) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Filter (26) - : : +- * HashAggregate (25) - : : +- Exchange (24) - : : +- * HashAggregate (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) +TakeOrderedAndProject (49) ++- * Project (48) + +- * BroadcastHashJoin Inner BuildRight (47) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Filter (16) + : : +- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (12) - : : : :- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (11) - : : : +- * Project (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.date_dim (7) - : : +- BroadcastExchange (20) - : : +- * Filter (19) - : : +- * ColumnarToRow (18) - : : +- Scan parquet default.item (17) - : +- BroadcastExchange (40) - : +- * Filter (39) - : +- * HashAggregate (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (32) - : : +- * BroadcastHashJoin Inner BuildRight (31) - : : :- * Filter (29) - : : : +- * ColumnarToRow (28) - : : : +- Scan parquet default.catalog_sales (27) - : : +- ReusedExchange (30) - : +- ReusedExchange (33) - +- BroadcastExchange (56) - +- * Filter (55) - +- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * Project (51) - +- * BroadcastHashJoin Inner BuildRight (50) - :- * Project (48) - : +- * BroadcastHashJoin Inner BuildRight (47) - : :- * Filter (45) - : : +- * ColumnarToRow (44) - : : +- Scan parquet default.web_sales (43) - : +- ReusedExchange (46) - +- ReusedExchange (49) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet default.item (7) + : +- BroadcastExchange (30) + : +- * Filter (29) + : +- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.catalog_sales (17) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (46) + +- * Filter (45) + +- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Filter (35) + : : +- * ColumnarToRow (34) + : : +- Scan parquet default.web_sales (33) + : +- ReusedExchange (36) + +- ReusedExchange (39) (1) Scan parquet default.store_sales Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct @@ -75,290 +65,311 @@ Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#4, d_date#5] - -(6) Filter [codegen id : 2] -Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) - -(7) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct +(4) ReusedExchange [Reuses operator id: 60] +Output [1]: [d_date_sk#5] -(8) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] - -(9) Filter [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] -Condition : (isnotnull(d_week_seq#7) AND (d_week_seq#7 = Subquery scalar-subquery#8, [id=#9])) - -(10) Project [codegen id : 1] -Output [1]: [d_date#6] -Input [2]: [d_date#6, d_week_seq#7] - -(11) BroadcastExchange -Input [1]: [d_date#6] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] - -(12) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_date#5] -Right keys [1]: [d_date#6] -Join condition: None - -(13) Project [codegen id : 2] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] - -(14) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(15) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#4] +Right keys [1]: [d_date_sk#5] Join condition: None -(16) Project [codegen id : 4] +(6) Project [codegen id : 4] Output [2]: [ss_item_sk#1, ss_ext_sales_price#2] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#4] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5] -(17) Scan parquet default.item -Output [2]: [i_item_sk#12, i_item_id#13] +(7) Scan parquet default.item +Output [2]: [i_item_sk#6, i_item_id#7] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#12, i_item_id#13] +(8) ColumnarToRow [codegen id : 3] +Input [2]: [i_item_sk#6, i_item_id#7] -(19) Filter [codegen id : 3] -Input [2]: [i_item_sk#12, i_item_id#13] -Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) +(9) Filter [codegen id : 3] +Input [2]: [i_item_sk#6, i_item_id#7] +Condition : (isnotnull(i_item_sk#6) AND isnotnull(i_item_id#7)) -(20) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] +(10) BroadcastExchange +Input [2]: [i_item_sk#6, i_item_id#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#8] -(21) BroadcastHashJoin [codegen id : 4] +(11) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#12] +Right keys [1]: [i_item_sk#6] Join condition: None -(22) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#2, i_item_id#13] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, i_item_sk#12, i_item_id#13] +(12) Project [codegen id : 4] +Output [2]: [ss_ext_sales_price#2, i_item_id#7] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, i_item_sk#6, i_item_id#7] -(23) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#2, i_item_id#13] -Keys [1]: [i_item_id#13] +(13) HashAggregate [codegen id : 4] +Input [2]: [ss_ext_sales_price#2, i_item_id#7] +Keys [1]: [i_item_id#7] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#13, sum#16] +Aggregate Attributes [1]: [sum#9] +Results [2]: [i_item_id#7, sum#10] -(24) Exchange -Input [2]: [i_item_id#13, sum#16] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] +(14) Exchange +Input [2]: [i_item_id#7, sum#10] +Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, [id=#11] -(25) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#13, sum#16] -Keys [1]: [i_item_id#13] +(15) HashAggregate [codegen id : 15] +Input [2]: [i_item_id#7, sum#10] +Keys [1]: [i_item_id#7] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#18] -Results [2]: [i_item_id#13 AS item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#18,17,2) AS ss_item_rev#20] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#12] +Results [2]: [i_item_id#7 AS item_id#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS ss_item_rev#14] -(26) Filter [codegen id : 15] -Input [2]: [item_id#19, ss_item_rev#20] -Condition : isnotnull(ss_item_rev#20) +(16) Filter [codegen id : 15] +Input [2]: [item_id#13, ss_item_rev#14] +Condition : isnotnull(ss_item_rev#14) -(27) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +(17) Scan parquet default.catalog_sales +Output [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#4)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +(18) ColumnarToRow [codegen id : 8] +Input [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] -(29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#21) +(19) Filter [codegen id : 8] +Input [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] +Condition : isnotnull(cs_item_sk#15) -(30) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#24] +(20) ReusedExchange [Reuses operator id: 60] +Output [1]: [d_date_sk#18] -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] +(21) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None -(32) Project [codegen id : 8] -Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] +(22) Project [codegen id : 8] +Output [2]: [cs_item_sk#15, cs_ext_sales_price#16] +Input [4]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17, d_date_sk#18] -(33) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#25, i_item_id#26] +(23) ReusedExchange [Reuses operator id: 10] +Output [2]: [i_item_sk#19, i_item_id#20] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#25] +(24) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#19] Join condition: None -(35) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#26] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#25, i_item_id#26] - -(36) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#26, sum#28] - -(37) Exchange -Input [2]: [i_item_id#26, sum#28] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [id=#29] - -(38) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#26, sum#28] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#26 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] - -(39) Filter [codegen id : 9] -Input [2]: [item_id#31, cs_item_rev#32] -Condition : isnotnull(cs_item_rev#32) - -(40) BroadcastExchange -Input [2]: [item_id#31, cs_item_rev#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] - -(41) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) - -(42) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] -Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] - -(43) Scan parquet default.web_sales -Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(25) Project [codegen id : 8] +Output [2]: [cs_ext_sales_price#16, i_item_id#20] +Input [4]: [cs_item_sk#15, cs_ext_sales_price#16, i_item_sk#19, i_item_id#20] + +(26) HashAggregate [codegen id : 8] +Input [2]: [cs_ext_sales_price#16, i_item_id#20] +Keys [1]: [i_item_id#20] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#16))] +Aggregate Attributes [1]: [sum#21] +Results [2]: [i_item_id#20, sum#22] + +(27) Exchange +Input [2]: [i_item_id#20, sum#22] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [id=#23] + +(28) HashAggregate [codegen id : 9] +Input [2]: [i_item_id#20, sum#22] +Keys [1]: [i_item_id#20] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#16))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#16))#24] +Results [2]: [i_item_id#20 AS item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#16))#24,17,2) AS cs_item_rev#26] + +(29) Filter [codegen id : 9] +Input [2]: [item_id#25, cs_item_rev#26] +Condition : isnotnull(cs_item_rev#26) + +(30) BroadcastExchange +Input [2]: [item_id#25, cs_item_rev#26] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#27] + +(31) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [item_id#13] +Right keys [1]: [item_id#25] +Join condition: ((((cast(ss_item_rev#14 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#26)), DecimalType(19,3), true)) AND (cast(ss_item_rev#14 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#26)), DecimalType(20,3), true))) AND (cast(cs_item_rev#26 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#14)), DecimalType(19,3), true))) AND (cast(cs_item_rev#26 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#14)), DecimalType(20,3), true))) + +(32) Project [codegen id : 15] +Output [3]: [item_id#13, ss_item_rev#14, cs_item_rev#26] +Input [4]: [item_id#13, ss_item_rev#14, item_id#25, cs_item_rev#26] + +(33) Scan parquet default.web_sales +Output [3]: [ws_item_sk#28, ws_ext_sales_price#29, ws_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(34) ColumnarToRow [codegen id : 13] +Input [3]: [ws_item_sk#28, ws_ext_sales_price#29, ws_sold_date_sk#30] -(45) Filter [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#34) +(35) Filter [codegen id : 13] +Input [3]: [ws_item_sk#28, ws_ext_sales_price#29, ws_sold_date_sk#30] +Condition : isnotnull(ws_item_sk#28) -(46) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#37] +(36) ReusedExchange [Reuses operator id: 60] +Output [1]: [d_date_sk#31] -(47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#37] +(37) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_sold_date_sk#30] +Right keys [1]: [d_date_sk#31] Join condition: None -(48) Project [codegen id : 13] -Output [2]: [ws_item_sk#34, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#37] +(38) Project [codegen id : 13] +Output [2]: [ws_item_sk#28, ws_ext_sales_price#29] +Input [4]: [ws_item_sk#28, ws_ext_sales_price#29, ws_sold_date_sk#30, d_date_sk#31] -(49) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#38, i_item_id#39] +(39) ReusedExchange [Reuses operator id: 10] +Output [2]: [i_item_sk#32, i_item_id#33] -(50) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#38] +(40) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_item_sk#28] +Right keys [1]: [i_item_sk#32] Join condition: None -(51) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#35, i_item_id#39] -Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, i_item_sk#38, i_item_id#39] - -(52) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#35, i_item_id#39] -Keys [1]: [i_item_id#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#39, sum#41] - -(53) Exchange -Input [2]: [i_item_id#39, sum#41] -Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, [id=#42] - -(54) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#39, sum#41] -Keys [1]: [i_item_id#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#39 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] - -(55) Filter [codegen id : 14] -Input [2]: [item_id#44, ws_item_rev#45] -Condition : isnotnull(ws_item_rev#45) - -(56) BroadcastExchange -Input [2]: [item_id#44, ws_item_rev#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] - -(57) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) - -(58) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] - -(59) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +(41) Project [codegen id : 13] +Output [2]: [ws_ext_sales_price#29, i_item_id#33] +Input [4]: [ws_item_sk#28, ws_ext_sales_price#29, i_item_sk#32, i_item_id#33] + +(42) HashAggregate [codegen id : 13] +Input [2]: [ws_ext_sales_price#29, i_item_id#33] +Keys [1]: [i_item_id#33] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#29))] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#33, sum#35] + +(43) Exchange +Input [2]: [i_item_id#33, sum#35] +Arguments: hashpartitioning(i_item_id#33, 5), ENSURE_REQUIREMENTS, [id=#36] + +(44) HashAggregate [codegen id : 14] +Input [2]: [i_item_id#33, sum#35] +Keys [1]: [i_item_id#33] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#29))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#29))#37] +Results [2]: [i_item_id#33 AS item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#29))#37,17,2) AS ws_item_rev#39] + +(45) Filter [codegen id : 14] +Input [2]: [item_id#38, ws_item_rev#39] +Condition : isnotnull(ws_item_rev#39) + +(46) BroadcastExchange +Input [2]: [item_id#38, ws_item_rev#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] + +(47) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [item_id#13] +Right keys [1]: [item_id#38] +Join condition: ((((((((cast(ss_item_rev#14 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#39)), DecimalType(19,3), true)) AND (cast(ss_item_rev#14 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#39)), DecimalType(20,3), true))) AND (cast(cs_item_rev#26 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#39)), DecimalType(19,3), true))) AND (cast(cs_item_rev#26 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#39)), DecimalType(20,3), true))) AND (cast(ws_item_rev#39 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#14)), DecimalType(19,3), true))) AND (cast(ws_item_rev#39 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#14)), DecimalType(20,3), true))) AND (cast(ws_item_rev#39 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#26)), DecimalType(19,3), true))) AND (cast(ws_item_rev#39 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#26)), DecimalType(20,3), true))) + +(48) Project [codegen id : 15] +Output [8]: [item_id#13, ss_item_rev#14, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#14 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#14 as decimal(18,2))) + promote_precision(cast(cs_item_rev#26 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#39 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#41, cs_item_rev#26, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#26 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#14 as decimal(18,2))) + promote_precision(cast(cs_item_rev#26 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#39 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#42, ws_item_rev#39, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#39 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#14 as decimal(18,2))) + promote_precision(cast(cs_item_rev#26 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#39 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#43, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#14 as decimal(18,2))) + promote_precision(cast(cs_item_rev#26 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#39 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#44] +Input [5]: [item_id#13, ss_item_rev#14, cs_item_rev#26, item_id#38, ws_item_rev#39] + +(49) TakeOrderedAndProject +Input [8]: [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44] +Arguments: 100, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44] ===== Subqueries ===== -Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* Project (63) -+- * Filter (62) - +- * ColumnarToRow (61) - +- Scan parquet default.date_dim (60) +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (60) ++- * Project (59) + +- * BroadcastHashJoin LeftSemi BuildRight (58) + :- * Filter (52) + : +- * ColumnarToRow (51) + : +- Scan parquet default.date_dim (50) + +- BroadcastExchange (57) + +- * Project (56) + +- * Filter (55) + +- * ColumnarToRow (54) + +- Scan parquet default.date_dim (53) + + +(50) Scan parquet default.date_dim +Output [2]: [d_date_sk#5, d_date#45] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct + +(51) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#5, d_date#45] + +(52) Filter [codegen id : 2] +Input [2]: [d_date_sk#5, d_date#45] +Condition : isnotnull(d_date_sk#5) + +(53) Scan parquet default.date_dim +Output [2]: [d_date#46, d_week_seq#47] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq)] +ReadSchema: struct + +(54) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#46, d_week_seq#47] +(55) Filter [codegen id : 1] +Input [2]: [d_date#46, d_week_seq#47] +Condition : (isnotnull(d_week_seq#47) AND (d_week_seq#47 = Subquery scalar-subquery#48, [id=#49])) -(60) Scan parquet default.date_dim -Output [2]: [d_date#51, d_week_seq#52] +(56) Project [codegen id : 1] +Output [1]: [d_date#46] +Input [2]: [d_date#46, d_week_seq#47] + +(57) BroadcastExchange +Input [1]: [d_date#46] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#50] + +(58) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [d_date#45] +Right keys [1]: [d_date#46] +Join condition: None + +(59) Project [codegen id : 2] +Output [1]: [d_date_sk#5] +Input [2]: [d_date_sk#5, d_date#45] + +(60) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] + +Subquery:2 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* Project (64) ++- * Filter (63) + +- * ColumnarToRow (62) + +- Scan parquet default.date_dim (61) + + +(61) Scan parquet default.date_dim +Output [2]: [d_date#52, d_week_seq#53] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#51, d_week_seq#52] +(62) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#52, d_week_seq#53] + +(63) Filter [codegen id : 1] +Input [2]: [d_date#52, d_week_seq#53] +Condition : (isnotnull(d_date#52) AND (d_date#52 = 2000-01-03)) + +(64) Project [codegen id : 1] +Output [1]: [d_week_seq#53] +Input [2]: [d_date#52, d_week_seq#53] -(62) Filter [codegen id : 1] -Input [2]: [d_date#51, d_week_seq#52] -Condition : (isnotnull(d_date#51) AND (d_date#51 = 2000-01-03)) +Subquery:3 Hosting operator id = 17 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#4 -(63) Project [codegen id : 1] -Output [1]: [d_week_seq#52] -Input [2]: [d_date#51, d_week_seq#52] +Subquery:4 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt index 57c79b9aee840..6f6eb5db635cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt @@ -18,30 +18,32 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date] - Filter [d_week_seq] - Subquery #1 + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (2) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + Filter [d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_date] + Project [d_date] + Filter [d_week_seq] + Subquery #2 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #4 WholeStageCodegen (3) @@ -66,6 +68,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter @@ -87,6 +90,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index a8ce33ee99d7b..67f19d31e3946 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -1,15 +1,15 @@ == Physical Plan == -TakeOrderedAndProject (59) -+- * Project (58) - +- * BroadcastHashJoin Inner BuildRight (57) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Filter (26) - : : +- * HashAggregate (25) - : : +- Exchange (24) - : : +- * HashAggregate (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) +TakeOrderedAndProject (49) ++- * Project (48) + +- * BroadcastHashJoin Inner BuildRight (47) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Filter (16) + : : +- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) : : :- * Project (9) : : : +- * BroadcastHashJoin Inner BuildRight (8) : : : :- * Filter (3) @@ -19,52 +19,42 @@ TakeOrderedAndProject (59) : : : +- * Filter (6) : : : +- * ColumnarToRow (5) : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin LeftSemi BuildRight (18) - : : :- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (17) - : : +- * Project (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.date_dim (13) - : +- BroadcastExchange (40) - : +- * Filter (39) - : +- * HashAggregate (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (32) - : : +- * BroadcastHashJoin Inner BuildRight (31) - : : :- * Filter (29) - : : : +- * ColumnarToRow (28) - : : : +- Scan parquet default.catalog_sales (27) - : : +- ReusedExchange (30) - : +- ReusedExchange (33) - +- BroadcastExchange (56) - +- * Filter (55) - +- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * Project (51) - +- * BroadcastHashJoin Inner BuildRight (50) - :- * Project (48) - : +- * BroadcastHashJoin Inner BuildRight (47) - : :- * Filter (45) - : : +- * ColumnarToRow (44) - : : +- Scan parquet default.web_sales (43) - : +- ReusedExchange (46) - +- ReusedExchange (49) + : : +- ReusedExchange (10) + : +- BroadcastExchange (30) + : +- * Filter (29) + : +- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.catalog_sales (17) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (46) + +- * Filter (45) + +- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Filter (35) + : : +- * ColumnarToRow (34) + : : +- Scan parquet default.web_sales (33) + : +- ReusedExchange (36) + +- ReusedExchange (39) (1) Scan parquet default.store_sales Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct @@ -76,289 +66,310 @@ Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) (4) Scan parquet default.item -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#5, i_item_id#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#4, i_item_id#5] +Input [2]: [i_item_sk#5, i_item_id#6] (6) Filter [codegen id : 1] -Input [2]: [i_item_sk#4, i_item_id#5] -Condition : (isnotnull(i_item_sk#4) AND isnotnull(i_item_id#5)) +Input [2]: [i_item_sk#5, i_item_id#6] +Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) (7) BroadcastExchange -Input [2]: [i_item_sk#4, i_item_id#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#6] +Input [2]: [i_item_sk#5, i_item_id#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] (8) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#4] +Right keys [1]: [i_item_sk#5] Join condition: None (9) Project [codegen id : 4] -Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#5] -Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#4, i_item_id#5] +Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6] +Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6] -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#7, d_date#8] - -(12) Filter [codegen id : 3] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) - -(13) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct +(10) ReusedExchange [Reuses operator id: 60] +Output [1]: [d_date_sk#8] -(14) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#9, d_week_seq#10] - -(15) Filter [codegen id : 2] -Input [2]: [d_date#9, d_week_seq#10] -Condition : (isnotnull(d_week_seq#10) AND (d_week_seq#10 = Subquery scalar-subquery#11, [id=#12])) - -(16) Project [codegen id : 2] -Output [1]: [d_date#9] -Input [2]: [d_date#9, d_week_seq#10] - -(17) BroadcastExchange -Input [1]: [d_date#9] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] - -(18) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#9] -Join condition: None - -(19) Project [codegen id : 3] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] - -(20) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] - -(21) BroadcastHashJoin [codegen id : 4] +(11) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#8] Join condition: None -(22) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#2, i_item_id#5] -Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#5, d_date_sk#7] +(12) Project [codegen id : 4] +Output [2]: [ss_ext_sales_price#2, i_item_id#6] +Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, d_date_sk#8] -(23) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#2, i_item_id#5] -Keys [1]: [i_item_id#5] +(13) HashAggregate [codegen id : 4] +Input [2]: [ss_ext_sales_price#2, i_item_id#6] +Keys [1]: [i_item_id#6] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#5, sum#16] +Aggregate Attributes [1]: [sum#9] +Results [2]: [i_item_id#6, sum#10] -(24) Exchange -Input [2]: [i_item_id#5, sum#16] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#17] +(14) Exchange +Input [2]: [i_item_id#6, sum#10] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#11] -(25) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#5, sum#16] -Keys [1]: [i_item_id#5] +(15) HashAggregate [codegen id : 15] +Input [2]: [i_item_id#6, sum#10] +Keys [1]: [i_item_id#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#18] -Results [2]: [i_item_id#5 AS item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#18,17,2) AS ss_item_rev#20] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#12] +Results [2]: [i_item_id#6 AS item_id#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS ss_item_rev#14] -(26) Filter [codegen id : 15] -Input [2]: [item_id#19, ss_item_rev#20] -Condition : isnotnull(ss_item_rev#20) +(16) Filter [codegen id : 15] +Input [2]: [item_id#13, ss_item_rev#14] +Condition : isnotnull(ss_item_rev#14) -(27) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +(17) Scan parquet default.catalog_sales +Output [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#4)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +(18) ColumnarToRow [codegen id : 8] +Input [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] -(29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#21) +(19) Filter [codegen id : 8] +Input [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] +Condition : isnotnull(cs_item_sk#15) -(30) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#24, i_item_id#25] +(20) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#18, i_item_id#19] -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#24] +(21) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#18] Join condition: None -(32) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25] -Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#24, i_item_id#25] +(22) Project [codegen id : 8] +Output [3]: [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#19] +Input [5]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_sk#18, i_item_id#19] -(33) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#26] +(23) ReusedExchange [Reuses operator id: 60] +Output [1]: [d_date_sk#20] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#26] +(24) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#20] Join condition: None -(35) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#25] -Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25, d_date_sk#26] - -(36) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#25] -Keys [1]: [i_item_id#25] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#25, sum#28] - -(37) Exchange -Input [2]: [i_item_id#25, sum#28] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#29] - -(38) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#25, sum#28] -Keys [1]: [i_item_id#25] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#25 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] - -(39) Filter [codegen id : 9] -Input [2]: [item_id#31, cs_item_rev#32] -Condition : isnotnull(cs_item_rev#32) - -(40) BroadcastExchange -Input [2]: [item_id#31, cs_item_rev#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] - -(41) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) - -(42) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] -Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] - -(43) Scan parquet default.web_sales -Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(25) Project [codegen id : 8] +Output [2]: [cs_ext_sales_price#16, i_item_id#19] +Input [4]: [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#19, d_date_sk#20] + +(26) HashAggregate [codegen id : 8] +Input [2]: [cs_ext_sales_price#16, i_item_id#19] +Keys [1]: [i_item_id#19] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#16))] +Aggregate Attributes [1]: [sum#21] +Results [2]: [i_item_id#19, sum#22] + +(27) Exchange +Input [2]: [i_item_id#19, sum#22] +Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, [id=#23] + +(28) HashAggregate [codegen id : 9] +Input [2]: [i_item_id#19, sum#22] +Keys [1]: [i_item_id#19] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#16))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#16))#24] +Results [2]: [i_item_id#19 AS item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#16))#24,17,2) AS cs_item_rev#26] + +(29) Filter [codegen id : 9] +Input [2]: [item_id#25, cs_item_rev#26] +Condition : isnotnull(cs_item_rev#26) + +(30) BroadcastExchange +Input [2]: [item_id#25, cs_item_rev#26] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#27] + +(31) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [item_id#13] +Right keys [1]: [item_id#25] +Join condition: ((((cast(ss_item_rev#14 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#26)), DecimalType(19,3), true)) AND (cast(ss_item_rev#14 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#26)), DecimalType(20,3), true))) AND (cast(cs_item_rev#26 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#14)), DecimalType(19,3), true))) AND (cast(cs_item_rev#26 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#14)), DecimalType(20,3), true))) + +(32) Project [codegen id : 15] +Output [3]: [item_id#13, ss_item_rev#14, cs_item_rev#26] +Input [4]: [item_id#13, ss_item_rev#14, item_id#25, cs_item_rev#26] + +(33) Scan parquet default.web_sales +Output [3]: [ws_item_sk#28, ws_ext_sales_price#29, ws_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(34) ColumnarToRow [codegen id : 13] +Input [3]: [ws_item_sk#28, ws_ext_sales_price#29, ws_sold_date_sk#30] -(45) Filter [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#34) +(35) Filter [codegen id : 13] +Input [3]: [ws_item_sk#28, ws_ext_sales_price#29, ws_sold_date_sk#30] +Condition : isnotnull(ws_item_sk#28) -(46) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#37, i_item_id#38] +(36) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#31, i_item_id#32] -(47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#37] +(37) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_item_sk#28] +Right keys [1]: [i_item_sk#31] Join condition: None -(48) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#38] -Input [5]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_sk#37, i_item_id#38] +(38) Project [codegen id : 13] +Output [3]: [ws_ext_sales_price#29, ws_sold_date_sk#30, i_item_id#32] +Input [5]: [ws_item_sk#28, ws_ext_sales_price#29, ws_sold_date_sk#30, i_item_sk#31, i_item_id#32] -(49) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#39] +(39) ReusedExchange [Reuses operator id: 60] +Output [1]: [d_date_sk#33] -(50) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#39] +(40) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_sold_date_sk#30] +Right keys [1]: [d_date_sk#33] Join condition: None -(51) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#35, i_item_id#38] -Input [4]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#38, d_date_sk#39] - -(52) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#35, i_item_id#38] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#38, sum#41] - -(53) Exchange -Input [2]: [i_item_id#38, sum#41] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [id=#42] - -(54) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#38, sum#41] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#38 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] - -(55) Filter [codegen id : 14] -Input [2]: [item_id#44, ws_item_rev#45] -Condition : isnotnull(ws_item_rev#45) - -(56) BroadcastExchange -Input [2]: [item_id#44, ws_item_rev#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] - -(57) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) - -(58) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] - -(59) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +(41) Project [codegen id : 13] +Output [2]: [ws_ext_sales_price#29, i_item_id#32] +Input [4]: [ws_ext_sales_price#29, ws_sold_date_sk#30, i_item_id#32, d_date_sk#33] + +(42) HashAggregate [codegen id : 13] +Input [2]: [ws_ext_sales_price#29, i_item_id#32] +Keys [1]: [i_item_id#32] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#29))] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#32, sum#35] + +(43) Exchange +Input [2]: [i_item_id#32, sum#35] +Arguments: hashpartitioning(i_item_id#32, 5), ENSURE_REQUIREMENTS, [id=#36] + +(44) HashAggregate [codegen id : 14] +Input [2]: [i_item_id#32, sum#35] +Keys [1]: [i_item_id#32] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#29))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#29))#37] +Results [2]: [i_item_id#32 AS item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#29))#37,17,2) AS ws_item_rev#39] + +(45) Filter [codegen id : 14] +Input [2]: [item_id#38, ws_item_rev#39] +Condition : isnotnull(ws_item_rev#39) + +(46) BroadcastExchange +Input [2]: [item_id#38, ws_item_rev#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] + +(47) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [item_id#13] +Right keys [1]: [item_id#38] +Join condition: ((((((((cast(ss_item_rev#14 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#39)), DecimalType(19,3), true)) AND (cast(ss_item_rev#14 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#39)), DecimalType(20,3), true))) AND (cast(cs_item_rev#26 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#39)), DecimalType(19,3), true))) AND (cast(cs_item_rev#26 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#39)), DecimalType(20,3), true))) AND (cast(ws_item_rev#39 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#14)), DecimalType(19,3), true))) AND (cast(ws_item_rev#39 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#14)), DecimalType(20,3), true))) AND (cast(ws_item_rev#39 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#26)), DecimalType(19,3), true))) AND (cast(ws_item_rev#39 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#26)), DecimalType(20,3), true))) + +(48) Project [codegen id : 15] +Output [8]: [item_id#13, ss_item_rev#14, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#14 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#14 as decimal(18,2))) + promote_precision(cast(cs_item_rev#26 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#39 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#41, cs_item_rev#26, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#26 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#14 as decimal(18,2))) + promote_precision(cast(cs_item_rev#26 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#39 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#42, ws_item_rev#39, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#39 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#14 as decimal(18,2))) + promote_precision(cast(cs_item_rev#26 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#39 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#43, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#14 as decimal(18,2))) + promote_precision(cast(cs_item_rev#26 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#39 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#44] +Input [5]: [item_id#13, ss_item_rev#14, cs_item_rev#26, item_id#38, ws_item_rev#39] + +(49) TakeOrderedAndProject +Input [8]: [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44] +Arguments: 100, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44] ===== Subqueries ===== -Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (63) -+- * Filter (62) - +- * ColumnarToRow (61) - +- Scan parquet default.date_dim (60) +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (60) ++- * Project (59) + +- * BroadcastHashJoin LeftSemi BuildRight (58) + :- * Filter (52) + : +- * ColumnarToRow (51) + : +- Scan parquet default.date_dim (50) + +- BroadcastExchange (57) + +- * Project (56) + +- * Filter (55) + +- * ColumnarToRow (54) + +- Scan parquet default.date_dim (53) + + +(50) Scan parquet default.date_dim +Output [2]: [d_date_sk#8, d_date#45] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct + +(51) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#8, d_date#45] + +(52) Filter [codegen id : 2] +Input [2]: [d_date_sk#8, d_date#45] +Condition : isnotnull(d_date_sk#8) + +(53) Scan parquet default.date_dim +Output [2]: [d_date#46, d_week_seq#47] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq)] +ReadSchema: struct + +(54) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#46, d_week_seq#47] +(55) Filter [codegen id : 1] +Input [2]: [d_date#46, d_week_seq#47] +Condition : (isnotnull(d_week_seq#47) AND (d_week_seq#47 = Subquery scalar-subquery#48, [id=#49])) -(60) Scan parquet default.date_dim -Output [2]: [d_date#51, d_week_seq#52] +(56) Project [codegen id : 1] +Output [1]: [d_date#46] +Input [2]: [d_date#46, d_week_seq#47] + +(57) BroadcastExchange +Input [1]: [d_date#46] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#50] + +(58) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [d_date#45] +Right keys [1]: [d_date#46] +Join condition: None + +(59) Project [codegen id : 2] +Output [1]: [d_date_sk#8] +Input [2]: [d_date_sk#8, d_date#45] + +(60) BroadcastExchange +Input [1]: [d_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] + +Subquery:2 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* Project (64) ++- * Filter (63) + +- * ColumnarToRow (62) + +- Scan parquet default.date_dim (61) + + +(61) Scan parquet default.date_dim +Output [2]: [d_date#52, d_week_seq#53] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#51, d_week_seq#52] +(62) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#52, d_week_seq#53] + +(63) Filter [codegen id : 1] +Input [2]: [d_date#52, d_week_seq#53] +Condition : (isnotnull(d_date#52) AND (d_date#52 = 2000-01-03)) + +(64) Project [codegen id : 1] +Output [1]: [d_week_seq#53] +Input [2]: [d_date#52, d_week_seq#53] -(62) Filter [codegen id : 1] -Input [2]: [d_date#51, d_week_seq#52] -Condition : (isnotnull(d_date#51) AND (d_date#51 = 2000-01-03)) +Subquery:3 Hosting operator id = 17 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#4 -(63) Project [codegen id : 1] -Output [1]: [d_week_seq#52] -Input [2]: [d_date#51, d_week_seq#52] +Subquery:4 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 1fb6832ea9b4a..cb69889cd4e7c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -18,37 +18,39 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (2) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + Filter [d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date] + Filter [d_week_seq] + Subquery #2 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter - BroadcastExchange #2 + BroadcastExchange #4 WholeStageCodegen (1) Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date] - Filter [d_week_seq] - Subquery #1 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 WholeStageCodegen (9) @@ -66,10 +68,11 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 + ReusedExchange [i_item_sk,i_item_id] #4 InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 WholeStageCodegen (14) @@ -87,7 +90,8 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 + ReusedExchange [i_item_sk,i_item_id] #4 InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #2 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt index b6b08c52ef055..e8c77e7de05eb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt @@ -101,182 +101,182 @@ Right keys [1]: [s_store_sk#6] Join condition: None (12) Project [codegen id : 3] -Output [3]: [ss_net_profit#2, s_store_name#7, substr(s_zip#8, 1, 2) AS substr(spark_catalog.default.store.s_zip, 1, 2)#10] +Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#8] (13) Exchange -Input [3]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10] -Arguments: hashpartitioning(substr(spark_catalog.default.store.s_zip, 1, 2)#10, 5), ENSURE_REQUIREMENTS, [id=#11] +Input [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] +Arguments: hashpartitioning(substr(s_zip#8, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#10] (14) Sort [codegen id : 4] -Input [3]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10] -Arguments: [substr(spark_catalog.default.store.s_zip, 1, 2)#10 ASC NULLS FIRST], false, 0 +Input [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] +Arguments: [substr(s_zip#8, 1, 2) ASC NULLS FIRST], false, 0 (15) Scan parquet default.customer_address -Output [1]: [ca_zip#12] +Output [1]: [ca_zip#11] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] ReadSchema: struct (16) ColumnarToRow [codegen id : 11] -Input [1]: [ca_zip#12] +Input [1]: [ca_zip#11] (17) Filter [codegen id : 11] -Input [1]: [ca_zip#12] -Condition : (substr(ca_zip#12, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#12, 1, 5))) +Input [1]: [ca_zip#11] +Condition : (substr(ca_zip#11, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#11, 1, 5))) (18) Scan parquet default.customer_address -Output [2]: [ca_address_sk#13, ca_zip#14] +Output [2]: [ca_address_sk#12, ca_zip#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#13, ca_zip#14] +Input [2]: [ca_address_sk#12, ca_zip#13] (20) Filter [codegen id : 5] -Input [2]: [ca_address_sk#13, ca_zip#14] -Condition : isnotnull(ca_address_sk#13) +Input [2]: [ca_address_sk#12, ca_zip#13] +Condition : isnotnull(ca_address_sk#12) (21) Exchange -Input [2]: [ca_address_sk#13, ca_zip#14] -Arguments: hashpartitioning(ca_address_sk#13, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [2]: [ca_address_sk#12, ca_zip#13] +Arguments: hashpartitioning(ca_address_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] (22) Sort [codegen id : 6] -Input [2]: [ca_address_sk#13, ca_zip#14] -Arguments: [ca_address_sk#13 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#12, ca_zip#13] +Arguments: [ca_address_sk#12 ASC NULLS FIRST], false, 0 (23) Scan parquet default.customer -Output [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 7] -Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] (25) Filter [codegen id : 7] -Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] -Condition : ((isnotnull(c_preferred_cust_flag#17) AND (c_preferred_cust_flag#17 = Y)) AND isnotnull(c_current_addr_sk#16)) +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Condition : ((isnotnull(c_preferred_cust_flag#16) AND (c_preferred_cust_flag#16 = Y)) AND isnotnull(c_current_addr_sk#15)) (26) Project [codegen id : 7] -Output [1]: [c_current_addr_sk#16] -Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Output [1]: [c_current_addr_sk#15] +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] (27) Exchange -Input [1]: [c_current_addr_sk#16] -Arguments: hashpartitioning(c_current_addr_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [1]: [c_current_addr_sk#15] +Arguments: hashpartitioning(c_current_addr_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] (28) Sort [codegen id : 8] -Input [1]: [c_current_addr_sk#16] -Arguments: [c_current_addr_sk#16 ASC NULLS FIRST], false, 0 +Input [1]: [c_current_addr_sk#15] +Arguments: [c_current_addr_sk#15 ASC NULLS FIRST], false, 0 (29) SortMergeJoin [codegen id : 9] -Left keys [1]: [ca_address_sk#13] -Right keys [1]: [c_current_addr_sk#16] +Left keys [1]: [ca_address_sk#12] +Right keys [1]: [c_current_addr_sk#15] Join condition: None (30) Project [codegen id : 9] -Output [1]: [ca_zip#14] -Input [3]: [ca_address_sk#13, ca_zip#14, c_current_addr_sk#16] +Output [1]: [ca_zip#13] +Input [3]: [ca_address_sk#12, ca_zip#13, c_current_addr_sk#15] (31) HashAggregate [codegen id : 9] -Input [1]: [ca_zip#14] -Keys [1]: [ca_zip#14] +Input [1]: [ca_zip#13] +Keys [1]: [ca_zip#13] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#19] -Results [2]: [ca_zip#14, count#20] +Aggregate Attributes [1]: [count#18] +Results [2]: [ca_zip#13, count#19] (32) Exchange -Input [2]: [ca_zip#14, count#20] -Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [2]: [ca_zip#13, count#19] +Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, [id=#20] (33) HashAggregate [codegen id : 10] -Input [2]: [ca_zip#14, count#20] -Keys [1]: [ca_zip#14] +Input [2]: [ca_zip#13, count#19] +Keys [1]: [ca_zip#13] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#22] -Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#23, count(1)#22 AS cnt#24] +Aggregate Attributes [1]: [count(1)#21] +Results [2]: [substr(ca_zip#13, 1, 5) AS ca_zip#22, count(1)#21 AS cnt#23] (34) Filter [codegen id : 10] -Input [2]: [ca_zip#23, cnt#24] -Condition : (cnt#24 > 10) +Input [2]: [ca_zip#22, cnt#23] +Condition : (cnt#23 > 10) (35) Project [codegen id : 10] -Output [1]: [ca_zip#23] -Input [2]: [ca_zip#23, cnt#24] +Output [1]: [ca_zip#22] +Input [2]: [ca_zip#22, cnt#23] (36) BroadcastExchange -Input [1]: [ca_zip#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#25] +Input [1]: [ca_zip#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#24] (37) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [coalesce(substr(ca_zip#12, 1, 5), ), isnull(substr(ca_zip#12, 1, 5))] -Right keys [2]: [coalesce(ca_zip#23, ), isnull(ca_zip#23)] +Left keys [2]: [coalesce(substr(ca_zip#11, 1, 5), ), isnull(substr(ca_zip#11, 1, 5))] +Right keys [2]: [coalesce(ca_zip#22, ), isnull(ca_zip#22)] Join condition: None (38) Project [codegen id : 11] -Output [1]: [substr(ca_zip#12, 1, 5) AS ca_zip#26] -Input [1]: [ca_zip#12] +Output [1]: [substr(ca_zip#11, 1, 5) AS ca_zip#25] +Input [1]: [ca_zip#11] (39) HashAggregate [codegen id : 11] -Input [1]: [ca_zip#26] -Keys [1]: [ca_zip#26] +Input [1]: [ca_zip#25] +Keys [1]: [ca_zip#25] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#26] +Results [1]: [ca_zip#25] (40) Exchange -Input [1]: [ca_zip#26] -Arguments: hashpartitioning(ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [1]: [ca_zip#25] +Arguments: hashpartitioning(ca_zip#25, 5), ENSURE_REQUIREMENTS, [id=#26] (41) HashAggregate [codegen id : 12] -Input [1]: [ca_zip#26] -Keys [1]: [ca_zip#26] +Input [1]: [ca_zip#25] +Keys [1]: [ca_zip#25] Functions: [] Aggregate Attributes: [] -Results [1]: [substr(ca_zip#26, 1, 2) AS substr(V1.ca_zip, 1, 2)#28] +Results [1]: [ca_zip#25] (42) Exchange -Input [1]: [substr(V1.ca_zip, 1, 2)#28] -Arguments: hashpartitioning(substr(V1.ca_zip, 1, 2)#28, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [1]: [ca_zip#25] +Arguments: hashpartitioning(substr(ca_zip#25, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#27] (43) Sort [codegen id : 13] -Input [1]: [substr(V1.ca_zip, 1, 2)#28] -Arguments: [substr(V1.ca_zip, 1, 2)#28 ASC NULLS FIRST], false, 0 +Input [1]: [ca_zip#25] +Arguments: [substr(ca_zip#25, 1, 2) ASC NULLS FIRST], false, 0 (44) SortMergeJoin [codegen id : 14] -Left keys [1]: [substr(spark_catalog.default.store.s_zip, 1, 2)#10] -Right keys [1]: [substr(V1.ca_zip, 1, 2)#28] +Left keys [1]: [substr(s_zip#8, 1, 2)] +Right keys [1]: [substr(ca_zip#25, 1, 2)] Join condition: None (45) Project [codegen id : 14] Output [2]: [ss_net_profit#2, s_store_name#7] -Input [4]: [ss_net_profit#2, s_store_name#7, substr(spark_catalog.default.store.s_zip, 1, 2)#10, substr(V1.ca_zip, 1, 2)#28] +Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#8, ca_zip#25] (46) HashAggregate [codegen id : 14] Input [2]: [ss_net_profit#2, s_store_name#7] Keys [1]: [s_store_name#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#30] -Results [2]: [s_store_name#7, sum#31] +Aggregate Attributes [1]: [sum#28] +Results [2]: [s_store_name#7, sum#29] (47) Exchange -Input [2]: [s_store_name#7, sum#31] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [2]: [s_store_name#7, sum#29] +Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [id=#30] (48) HashAggregate [codegen id : 15] -Input [2]: [s_store_name#7, sum#31] +Input [2]: [s_store_name#7, sum#29] Keys [1]: [s_store_name#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#33] -Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#33,17,2) AS sum(ss_net_profit)#34] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] +Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS sum(ss_net_profit)#32] (49) TakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#34] -Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#34] +Input [2]: [s_store_name#7, sum(ss_net_profit)#32] +Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#32] ===== Subqueries ===== @@ -289,25 +289,25 @@ BroadcastExchange (54) (50) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#35, d_qoy#36] +Output [3]: [d_date_sk#5, d_year#33, d_qoy#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#35, d_qoy#36] +Input [3]: [d_date_sk#5, d_year#33, d_qoy#34] (52) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#35, d_qoy#36] -Condition : ((((isnotnull(d_qoy#36) AND isnotnull(d_year#35)) AND (d_qoy#36 = 2)) AND (d_year#35 = 1998)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#5, d_year#33, d_qoy#34] +Condition : ((((isnotnull(d_qoy#34) AND isnotnull(d_year#33)) AND (d_qoy#34 = 2)) AND (d_year#33 = 1998)) AND isnotnull(d_date_sk#5)) (53) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#35, d_qoy#36] +Input [3]: [d_date_sk#5, d_year#33, d_qoy#34] (54) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt index 435254f45f10d..84ac2edd606cf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt @@ -6,12 +6,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (14) HashAggregate [s_store_name,ss_net_profit] [sum,sum] Project [ss_net_profit,s_store_name] - SortMergeJoin [substr(spark_catalog.default.store.s_zip, 1, 2),substr(V1.ca_zip, 1, 2)] + SortMergeJoin [s_zip,ca_zip] InputAdapter WholeStageCodegen (4) - Sort [substr(spark_catalog.default.store.s_zip, 1, 2)] + Sort [s_zip] InputAdapter - Exchange [substr(spark_catalog.default.store.s_zip, 1, 2)] #2 + Exchange [s_zip] #2 WholeStageCodegen (3) Project [ss_net_profit,s_store_name,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -40,11 +40,11 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] Scan parquet default.store [s_store_sk,s_store_name,s_zip] InputAdapter WholeStageCodegen (13) - Sort [substr(V1.ca_zip, 1, 2)] + Sort [ca_zip] InputAdapter - Exchange [substr(V1.ca_zip, 1, 2)] #5 + Exchange [ca_zip] #5 WholeStageCodegen (12) - HashAggregate [ca_zip] [substr(V1.ca_zip, 1, 2)] + HashAggregate [ca_zip] InputAdapter Exchange [ca_zip] #6 WholeStageCodegen (11) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index 7f0868b3b4db2..175a1c675675f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -1,72 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (61) -+- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * HashAggregate (30) - : : +- Exchange (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) +TakeOrderedAndProject (46) ++- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_returns (1) - : : : +- BroadcastExchange (19) - : : : +- * Project (18) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (17) - : : : :- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (14) - : : : :- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (13) - : : : +- * Project (12) - : : : +- * Filter (11) - : : : +- * ColumnarToRow (10) - : : : +- Scan parquet default.date_dim (9) - : : +- BroadcastExchange (25) - : : +- * Filter (24) - : : +- * ColumnarToRow (23) - : : +- Scan parquet default.item (22) - : +- BroadcastExchange (43) - : +- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Filter (33) - : : : +- * ColumnarToRow (32) - : : : +- Scan parquet default.catalog_returns (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- BroadcastExchange (58) - +- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet default.web_returns (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet default.item (7) + : +- BroadcastExchange (28) + : +- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * Filter (18) + : : : +- * ColumnarToRow (17) + : : : +- Scan parquet default.catalog_returns (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- BroadcastExchange (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Filter (33) + : : +- * ColumnarToRow (32) + : : +- Scan parquet default.web_returns (31) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet default.store_returns Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(sr_item_sk)] ReadSchema: struct @@ -77,271 +62,300 @@ Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#4, d_date#5] - -(6) Filter [codegen id : 3] -Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) - -(7) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#6, d_week_seq#7] - -(9) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(10) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] +(4) ReusedExchange [Reuses operator id: 62] +Output [1]: [d_date_sk#5] -(11) Filter [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] -Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) - -(12) Project [codegen id : 1] -Output [1]: [d_week_seq#9] -Input [2]: [d_date#8, d_week_seq#9] - -(13) BroadcastExchange -Input [1]: [d_week_seq#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] - -(14) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#7] -Right keys [1]: [d_week_seq#9] -Join condition: None - -(15) Project [codegen id : 2] -Output [1]: [d_date#6] -Input [2]: [d_date#6, d_week_seq#7] - -(16) BroadcastExchange -Input [1]: [d_date#6] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#11] - -(17) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date#5] -Right keys [1]: [d_date#6] -Join condition: None - -(18) Project [codegen id : 3] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] - -(19) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(20) BroadcastHashJoin [codegen id : 5] +(5) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] -Right keys [1]: [d_date_sk#4] +Right keys [1]: [d_date_sk#5] Join condition: None -(21) Project [codegen id : 5] +(6) Project [codegen id : 5] Output [2]: [sr_item_sk#1, sr_return_quantity#2] -Input [4]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, d_date_sk#4] +Input [4]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, d_date_sk#5] -(22) Scan parquet default.item -Output [2]: [i_item_sk#13, i_item_id#14] +(7) Scan parquet default.item +Output [2]: [i_item_sk#6, i_item_id#7] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#13, i_item_id#14] +(8) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#6, i_item_id#7] -(24) Filter [codegen id : 4] -Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +(9) Filter [codegen id : 4] +Input [2]: [i_item_sk#6, i_item_id#7] +Condition : (isnotnull(i_item_sk#6) AND isnotnull(i_item_id#7)) -(25) BroadcastExchange -Input [2]: [i_item_sk#13, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] +(10) BroadcastExchange +Input [2]: [i_item_sk#6, i_item_id#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#8] -(26) BroadcastHashJoin [codegen id : 5] +(11) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#1] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#6] Join condition: None -(27) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#14] -Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#13, i_item_id#14] +(12) Project [codegen id : 5] +Output [2]: [sr_return_quantity#2, i_item_id#7] +Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#6, i_item_id#7] -(28) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#2, i_item_id#14] -Keys [1]: [i_item_id#14] +(13) HashAggregate [codegen id : 5] +Input [2]: [sr_return_quantity#2, i_item_id#7] +Keys [1]: [i_item_id#7] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#14, sum#17] +Aggregate Attributes [1]: [sum#9] +Results [2]: [i_item_id#7, sum#10] -(29) Exchange -Input [2]: [i_item_id#14, sum#17] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#18] +(14) Exchange +Input [2]: [i_item_id#7, sum#10] +Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, [id=#11] -(30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#17] -Keys [1]: [i_item_id#14] +(15) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#7, sum#10] +Keys [1]: [i_item_id#7] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] -Results [2]: [i_item_id#14 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#12] +Results [2]: [i_item_id#7 AS item_id#13, sum(sr_return_quantity#2)#12 AS sr_item_qty#14] -(31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +(16) Scan parquet default.catalog_returns +Output [3]: [cr_item_sk#15, cr_return_quantity#16, cr_returned_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cr_returned_date_sk#17), dynamicpruningexpression(cr_returned_date_sk#17 IN dynamicpruning#4)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +(17) ColumnarToRow [codegen id : 10] +Input [3]: [cr_item_sk#15, cr_return_quantity#16, cr_returned_date_sk#17] + +(18) Filter [codegen id : 10] +Input [3]: [cr_item_sk#15, cr_return_quantity#16, cr_returned_date_sk#17] +Condition : isnotnull(cr_item_sk#15) + +(19) ReusedExchange [Reuses operator id: 62] +Output [1]: [d_date_sk#18] + +(20) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#17] +Right keys [1]: [d_date_sk#18] +Join condition: None + +(21) Project [codegen id : 10] +Output [2]: [cr_item_sk#15, cr_return_quantity#16] +Input [4]: [cr_item_sk#15, cr_return_quantity#16, cr_returned_date_sk#17, d_date_sk#18] -(33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] -Condition : isnotnull(cr_item_sk#22) +(22) ReusedExchange [Reuses operator id: 10] +Output [2]: [i_item_sk#19, i_item_id#20] -(34) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#25] +(23) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_item_sk#15] +Right keys [1]: [i_item_sk#19] +Join condition: None + +(24) Project [codegen id : 10] +Output [2]: [cr_return_quantity#16, i_item_id#20] +Input [4]: [cr_item_sk#15, cr_return_quantity#16, i_item_sk#19, i_item_id#20] + +(25) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#16, i_item_id#20] +Keys [1]: [i_item_id#20] +Functions [1]: [partial_sum(cr_return_quantity#16)] +Aggregate Attributes [1]: [sum#21] +Results [2]: [i_item_id#20, sum#22] + +(26) Exchange +Input [2]: [i_item_id#20, sum#22] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [id=#23] + +(27) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#20, sum#22] +Keys [1]: [i_item_id#20] +Functions [1]: [sum(cr_return_quantity#16)] +Aggregate Attributes [1]: [sum(cr_return_quantity#16)#24] +Results [2]: [i_item_id#20 AS item_id#25, sum(cr_return_quantity#16)#24 AS cr_item_qty#26] + +(28) BroadcastExchange +Input [2]: [item_id#25, cr_item_qty#26] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#27] + +(29) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#13] +Right keys [1]: [item_id#25] +Join condition: None + +(30) Project [codegen id : 18] +Output [3]: [item_id#13, sr_item_qty#14, cr_item_qty#26] +Input [4]: [item_id#13, sr_item_qty#14, item_id#25, cr_item_qty#26] + +(31) Scan parquet default.web_returns +Output [3]: [wr_item_sk#28, wr_return_quantity#29, wr_returned_date_sk#30] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(wr_returned_date_sk#30), dynamicpruningexpression(wr_returned_date_sk#30 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(wr_item_sk)] +ReadSchema: struct + +(32) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#28, wr_return_quantity#29, wr_returned_date_sk#30] + +(33) Filter [codegen id : 16] +Input [3]: [wr_item_sk#28, wr_return_quantity#29, wr_returned_date_sk#30] +Condition : isnotnull(wr_item_sk#28) + +(34) ReusedExchange [Reuses operator id: 62] +Output [1]: [d_date_sk#31] -(35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#24] -Right keys [1]: [d_date_sk#25] +(35) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#30] +Right keys [1]: [d_date_sk#31] Join condition: None -(36) Project [codegen id : 10] -Output [2]: [cr_item_sk#22, cr_return_quantity#23] -Input [4]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, d_date_sk#25] +(36) Project [codegen id : 16] +Output [2]: [wr_item_sk#28, wr_return_quantity#29] +Input [4]: [wr_item_sk#28, wr_return_quantity#29, wr_returned_date_sk#30, d_date_sk#31] -(37) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#26, i_item_id#27] +(37) ReusedExchange [Reuses operator id: 10] +Output [2]: [i_item_sk#32, i_item_id#33] -(38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#22] -Right keys [1]: [i_item_sk#26] +(38) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#28] +Right keys [1]: [i_item_sk#32] Join condition: None -(39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#23, i_item_id#27] -Input [4]: [cr_item_sk#22, cr_return_quantity#23, i_item_sk#26, i_item_id#27] +(39) Project [codegen id : 16] +Output [2]: [wr_return_quantity#29, i_item_id#33] +Input [4]: [wr_item_sk#28, wr_return_quantity#29, i_item_sk#32, i_item_id#33] -(40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#23, i_item_id#27] -Keys [1]: [i_item_id#27] -Functions [1]: [partial_sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#27, sum#29] +(40) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#29, i_item_id#33] +Keys [1]: [i_item_id#33] +Functions [1]: [partial_sum(wr_return_quantity#29)] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#33, sum#35] (41) Exchange -Input [2]: [i_item_id#27, sum#29] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_item_id#33, sum#35] +Arguments: hashpartitioning(i_item_id#33, 5), ENSURE_REQUIREMENTS, [id=#36] -(42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#27, sum#29] -Keys [1]: [i_item_id#27] -Functions [1]: [sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum(cr_return_quantity#23)#31] -Results [2]: [i_item_id#27 AS item_id#32, sum(cr_return_quantity#23)#31 AS cr_item_qty#33] +(42) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#33, sum#35] +Keys [1]: [i_item_id#33] +Functions [1]: [sum(wr_return_quantity#29)] +Aggregate Attributes [1]: [sum(wr_return_quantity#29)#37] +Results [2]: [i_item_id#33 AS item_id#38, sum(wr_return_quantity#29)#37 AS wr_item_qty#39] (43) BroadcastExchange -Input [2]: [item_id#32, cr_item_qty#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] +Input [2]: [item_id#38, wr_item_qty#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] (44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#32] +Left keys [1]: [item_id#13] +Right keys [1]: [item_id#38] Join condition: None (45) Project [codegen id : 18] -Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#33] -Input [4]: [item_id#20, sr_item_qty#21, item_id#32, cr_item_qty#33] +Output [8]: [item_id#13, sr_item_qty#14, (((cast(sr_item_qty#14 as double) / cast(((sr_item_qty#14 + cr_item_qty#26) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#26, (((cast(cr_item_qty#26 as double) / cast(((sr_item_qty#14 + cr_item_qty#26) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#14 + cr_item_qty#26) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#14 + cr_item_qty#26) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] +Input [5]: [item_id#13, sr_item_qty#14, cr_item_qty#26, item_id#38, wr_item_qty#39] + +(46) TakeOrderedAndProject +Input [8]: [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Arguments: 100, [item_id#13 ASC NULLS FIRST, sr_item_qty#14 ASC NULLS FIRST], [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (62) ++- * Project (61) + +- * BroadcastHashJoin LeftSemi BuildRight (60) + :- * Filter (49) + : +- * ColumnarToRow (48) + : +- Scan parquet default.date_dim (47) + +- BroadcastExchange (59) + +- * Project (58) + +- * BroadcastHashJoin LeftSemi BuildRight (57) + :- * ColumnarToRow (51) + : +- Scan parquet default.date_dim (50) + +- BroadcastExchange (56) + +- * Project (55) + +- * Filter (54) + +- * ColumnarToRow (53) + +- Scan parquet default.date_dim (52) + + +(47) Scan parquet default.date_dim +Output [2]: [d_date_sk#5, d_date#45] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct -(46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +(48) ColumnarToRow [codegen id : 3] +Input [2]: [d_date_sk#5, d_date#45] + +(49) Filter [codegen id : 3] +Input [2]: [d_date_sk#5, d_date#45] +Condition : isnotnull(d_date_sk#5) + +(50) Scan parquet default.date_dim +Output [2]: [d_date#46, d_week_seq#47] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#37), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(wr_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +ReadSchema: struct + +(51) ColumnarToRow [codegen id : 2] +Input [2]: [d_date#46, d_week_seq#47] -(47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +(52) Scan parquet default.date_dim +Output [2]: [d_date#48, d_week_seq#49] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +ReadSchema: struct + +(53) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#48, d_week_seq#49] -(48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] -Condition : isnotnull(wr_item_sk#35) +(54) Filter [codegen id : 1] +Input [2]: [d_date#48, d_week_seq#49] +Condition : cast(d_date#48 as string) IN (2000-06-30,2000-09-27,2000-11-17) -(49) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#38] +(55) Project [codegen id : 1] +Output [1]: [d_week_seq#49] +Input [2]: [d_date#48, d_week_seq#49] -(50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#37] -Right keys [1]: [d_date_sk#38] +(56) BroadcastExchange +Input [1]: [d_week_seq#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] + +(57) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [d_week_seq#47] +Right keys [1]: [d_week_seq#49] Join condition: None -(51) Project [codegen id : 16] -Output [2]: [wr_item_sk#35, wr_return_quantity#36] -Input [4]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37, d_date_sk#38] +(58) Project [codegen id : 2] +Output [1]: [d_date#46] +Input [2]: [d_date#46, d_week_seq#47] -(52) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#39, i_item_id#40] +(59) BroadcastExchange +Input [1]: [d_date#46] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#51] -(53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#35] -Right keys [1]: [i_item_sk#39] +(60) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date#45] +Right keys [1]: [d_date#46] Join condition: None -(54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#36, i_item_id#40] -Input [4]: [wr_item_sk#35, wr_return_quantity#36, i_item_sk#39, i_item_id#40] - -(55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#36, i_item_id#40] -Keys [1]: [i_item_id#40] -Functions [1]: [partial_sum(wr_return_quantity#36)] -Aggregate Attributes [1]: [sum#41] -Results [2]: [i_item_id#40, sum#42] - -(56) Exchange -Input [2]: [i_item_id#40, sum#42] -Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, [id=#43] - -(57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#40, sum#42] -Keys [1]: [i_item_id#40] -Functions [1]: [sum(wr_return_quantity#36)] -Aggregate Attributes [1]: [sum(wr_return_quantity#36)#44] -Results [2]: [i_item_id#40 AS item_id#45, sum(wr_return_quantity#36)#44 AS wr_item_qty#46] - -(58) BroadcastExchange -Input [2]: [item_id#45, wr_item_qty#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] - -(59) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#45] -Join condition: None +(61) Project [codegen id : 3] +Output [1]: [d_date_sk#5] +Input [2]: [d_date_sk#5, d_date#45] + +(62) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] + +Subquery:2 Hosting operator id = 16 Hosting Expression = cr_returned_date_sk#17 IN dynamicpruning#4 -(60) Project [codegen id : 18] -Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS sr_dev#48, cr_item_qty#33, (((cast(cr_item_qty#33 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS cr_dev#49, wr_item_qty#46, (((cast(wr_item_qty#46 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS wr_dev#50, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#51] -Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#33, item_id#45, wr_item_qty#46] +Subquery:3 Hosting operator id = 31 Hosting Expression = wr_returned_date_sk#30 IN dynamicpruning#4 -(61) TakeOrderedAndProject -Input [8]: [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] -Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt index b46131afd5ec7..7f38503363767 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt @@ -17,31 +17,33 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [d_date] - BroadcastHashJoin [d_week_seq,d_week_seq] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (3) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [d_date] + BroadcastHashJoin [d_week_seq,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 WholeStageCodegen (4) @@ -65,6 +67,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter @@ -85,6 +88,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index 0764824706376..8332d48905e48 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,14 +1,14 @@ == Physical Plan == -TakeOrderedAndProject (61) -+- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * HashAggregate (30) - : : +- Exchange (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) +TakeOrderedAndProject (46) ++- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) : : :- * Project (9) : : : +- * BroadcastHashJoin Inner BuildRight (8) : : : :- * Filter (3) @@ -18,55 +18,40 @@ TakeOrderedAndProject (61) : : : +- * Filter (6) : : : +- * ColumnarToRow (5) : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- * BroadcastHashJoin LeftSemi BuildRight (20) - : : :- * ColumnarToRow (14) - : : : +- Scan parquet default.date_dim (13) - : : +- BroadcastExchange (19) - : : +- * Project (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.date_dim (15) - : +- BroadcastExchange (43) - : +- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Filter (33) - : : : +- * ColumnarToRow (32) - : : : +- Scan parquet default.catalog_returns (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- BroadcastExchange (58) - +- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet default.web_returns (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) + : : +- ReusedExchange (10) + : +- BroadcastExchange (28) + : +- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * Filter (18) + : : : +- * ColumnarToRow (17) + : : : +- Scan parquet default.catalog_returns (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- BroadcastExchange (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Filter (33) + : : +- * ColumnarToRow (32) + : : +- Scan parquet default.web_returns (31) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet default.store_returns Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(sr_item_sk)] ReadSchema: struct @@ -78,270 +63,299 @@ Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) (4) Scan parquet default.item -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#5, i_item_id#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#4, i_item_id#5] +Input [2]: [i_item_sk#5, i_item_id#6] (6) Filter [codegen id : 1] -Input [2]: [i_item_sk#4, i_item_id#5] -Condition : (isnotnull(i_item_sk#4) AND isnotnull(i_item_id#5)) +Input [2]: [i_item_sk#5, i_item_id#6] +Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) (7) BroadcastExchange -Input [2]: [i_item_sk#4, i_item_id#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#6] +Input [2]: [i_item_sk#5, i_item_id#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] (8) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#1] -Right keys [1]: [i_item_sk#4] +Right keys [1]: [i_item_sk#5] Join condition: None (9) Project [codegen id : 5] -Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#5] -Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#4, i_item_id#5] +Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] +Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#6] -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct +(10) ReusedExchange [Reuses operator id: 62] +Output [1]: [d_date_sk#8] -(11) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#7, d_date#8] +(11) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [sr_returned_date_sk#3] +Right keys [1]: [d_date_sk#8] +Join condition: None -(12) Filter [codegen id : 4] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +(12) Project [codegen id : 5] +Output [2]: [sr_return_quantity#2, i_item_id#6] +Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6, d_date_sk#8] -(13) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct +(13) HashAggregate [codegen id : 5] +Input [2]: [sr_return_quantity#2, i_item_id#6] +Keys [1]: [i_item_id#6] +Functions [1]: [partial_sum(sr_return_quantity#2)] +Aggregate Attributes [1]: [sum#9] +Results [2]: [i_item_id#6, sum#10] -(14) ColumnarToRow [codegen id : 3] -Input [2]: [d_date#9, d_week_seq#10] +(14) Exchange +Input [2]: [i_item_id#6, sum#10] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#11] -(15) Scan parquet default.date_dim -Output [2]: [d_date#11, d_week_seq#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct +(15) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#6, sum#10] +Keys [1]: [i_item_id#6] +Functions [1]: [sum(sr_return_quantity#2)] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#12] +Results [2]: [i_item_id#6 AS item_id#13, sum(sr_return_quantity#2)#12 AS sr_item_qty#14] -(16) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#11, d_week_seq#12] +(16) Scan parquet default.catalog_returns +Output [3]: [cr_item_sk#15, cr_return_quantity#16, cr_returned_date_sk#17] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cr_returned_date_sk#17), dynamicpruningexpression(cr_returned_date_sk#17 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(cr_item_sk)] +ReadSchema: struct -(17) Filter [codegen id : 2] -Input [2]: [d_date#11, d_week_seq#12] -Condition : cast(d_date#11 as string) IN (2000-06-30,2000-09-27,2000-11-17) +(17) ColumnarToRow [codegen id : 10] +Input [3]: [cr_item_sk#15, cr_return_quantity#16, cr_returned_date_sk#17] -(18) Project [codegen id : 2] -Output [1]: [d_week_seq#12] -Input [2]: [d_date#11, d_week_seq#12] +(18) Filter [codegen id : 10] +Input [3]: [cr_item_sk#15, cr_return_quantity#16, cr_returned_date_sk#17] +Condition : isnotnull(cr_item_sk#15) -(19) BroadcastExchange -Input [1]: [d_week_seq#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +(19) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#18, i_item_id#19] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#12] +(20) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_item_sk#15] +Right keys [1]: [i_item_sk#18] Join condition: None -(21) Project [codegen id : 3] -Output [1]: [d_date#9] -Input [2]: [d_date#9, d_week_seq#10] +(21) Project [codegen id : 10] +Output [3]: [cr_return_quantity#16, cr_returned_date_sk#17, i_item_id#19] +Input [5]: [cr_item_sk#15, cr_return_quantity#16, cr_returned_date_sk#17, i_item_sk#18, i_item_id#19] -(22) BroadcastExchange -Input [1]: [d_date#9] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#14] +(22) ReusedExchange [Reuses operator id: 62] +Output [1]: [d_date_sk#20] -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#9] +(23) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#17] +Right keys [1]: [d_date_sk#20] Join condition: None -(24) Project [codegen id : 4] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] - -(25) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sr_returned_date_sk#3] -Right keys [1]: [d_date_sk#7] +(24) Project [codegen id : 10] +Output [2]: [cr_return_quantity#16, i_item_id#19] +Input [4]: [cr_return_quantity#16, cr_returned_date_sk#17, i_item_id#19, d_date_sk#20] + +(25) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#16, i_item_id#19] +Keys [1]: [i_item_id#19] +Functions [1]: [partial_sum(cr_return_quantity#16)] +Aggregate Attributes [1]: [sum#21] +Results [2]: [i_item_id#19, sum#22] + +(26) Exchange +Input [2]: [i_item_id#19, sum#22] +Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, [id=#23] + +(27) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#19, sum#22] +Keys [1]: [i_item_id#19] +Functions [1]: [sum(cr_return_quantity#16)] +Aggregate Attributes [1]: [sum(cr_return_quantity#16)#24] +Results [2]: [i_item_id#19 AS item_id#25, sum(cr_return_quantity#16)#24 AS cr_item_qty#26] + +(28) BroadcastExchange +Input [2]: [item_id#25, cr_item_qty#26] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#27] + +(29) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#13] +Right keys [1]: [item_id#25] Join condition: None -(27) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#5] -Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#5, d_date_sk#7] - -(28) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#2, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#5, sum#17] - -(29) Exchange -Input [2]: [i_item_id#5, sum#17] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#18] - -(30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#5, sum#17] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] -Results [2]: [i_item_id#5 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] +(30) Project [codegen id : 18] +Output [3]: [item_id#13, sr_item_qty#14, cr_item_qty#26] +Input [4]: [item_id#13, sr_item_qty#14, item_id#25, cr_item_qty#26] -(31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +(31) Scan parquet default.web_returns +Output [3]: [wr_item_sk#28, wr_return_quantity#29, wr_returned_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(cr_item_sk)] -ReadSchema: struct +PartitionFilters: [isnotnull(wr_returned_date_sk#30), dynamicpruningexpression(wr_returned_date_sk#30 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(wr_item_sk)] +ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +(32) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#28, wr_return_quantity#29, wr_returned_date_sk#30] -(33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] -Condition : isnotnull(cr_item_sk#22) +(33) Filter [codegen id : 16] +Input [3]: [wr_item_sk#28, wr_return_quantity#29, wr_returned_date_sk#30] +Condition : isnotnull(wr_item_sk#28) (34) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#25, i_item_id#26] +Output [2]: [i_item_sk#31, i_item_id#32] -(35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#22] -Right keys [1]: [i_item_sk#25] +(35) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#28] +Right keys [1]: [i_item_sk#31] Join condition: None -(36) Project [codegen id : 10] -Output [3]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#26] -Input [5]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, i_item_sk#25, i_item_id#26] +(36) Project [codegen id : 16] +Output [3]: [wr_return_quantity#29, wr_returned_date_sk#30, i_item_id#32] +Input [5]: [wr_item_sk#28, wr_return_quantity#29, wr_returned_date_sk#30, i_item_sk#31, i_item_id#32] -(37) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#27] +(37) ReusedExchange [Reuses operator id: 62] +Output [1]: [d_date_sk#33] -(38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#24] -Right keys [1]: [d_date_sk#27] +(38) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#30] +Right keys [1]: [d_date_sk#33] Join condition: None -(39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#23, i_item_id#26] -Input [4]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#26, d_date_sk#27] +(39) Project [codegen id : 16] +Output [2]: [wr_return_quantity#29, i_item_id#32] +Input [4]: [wr_return_quantity#29, wr_returned_date_sk#30, i_item_id#32, d_date_sk#33] -(40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#23, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#26, sum#29] +(40) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#29, i_item_id#32] +Keys [1]: [i_item_id#32] +Functions [1]: [partial_sum(wr_return_quantity#29)] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#32, sum#35] (41) Exchange -Input [2]: [i_item_id#26, sum#29] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_item_id#32, sum#35] +Arguments: hashpartitioning(i_item_id#32, 5), ENSURE_REQUIREMENTS, [id=#36] -(42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#26, sum#29] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum(cr_return_quantity#23)#31] -Results [2]: [i_item_id#26 AS item_id#32, sum(cr_return_quantity#23)#31 AS cr_item_qty#33] +(42) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#32, sum#35] +Keys [1]: [i_item_id#32] +Functions [1]: [sum(wr_return_quantity#29)] +Aggregate Attributes [1]: [sum(wr_return_quantity#29)#37] +Results [2]: [i_item_id#32 AS item_id#38, sum(wr_return_quantity#29)#37 AS wr_item_qty#39] (43) BroadcastExchange -Input [2]: [item_id#32, cr_item_qty#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] +Input [2]: [item_id#38, wr_item_qty#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] (44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#32] +Left keys [1]: [item_id#13] +Right keys [1]: [item_id#38] Join condition: None (45) Project [codegen id : 18] -Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#33] -Input [4]: [item_id#20, sr_item_qty#21, item_id#32, cr_item_qty#33] +Output [8]: [item_id#13, sr_item_qty#14, (((cast(sr_item_qty#14 as double) / cast(((sr_item_qty#14 + cr_item_qty#26) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#26, (((cast(cr_item_qty#26 as double) / cast(((sr_item_qty#14 + cr_item_qty#26) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#14 + cr_item_qty#26) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#14 + cr_item_qty#26) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] +Input [5]: [item_id#13, sr_item_qty#14, cr_item_qty#26, item_id#38, wr_item_qty#39] + +(46) TakeOrderedAndProject +Input [8]: [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Arguments: 100, [item_id#13 ASC NULLS FIRST, sr_item_qty#14 ASC NULLS FIRST], [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (62) ++- * Project (61) + +- * BroadcastHashJoin LeftSemi BuildRight (60) + :- * Filter (49) + : +- * ColumnarToRow (48) + : +- Scan parquet default.date_dim (47) + +- BroadcastExchange (59) + +- * Project (58) + +- * BroadcastHashJoin LeftSemi BuildRight (57) + :- * ColumnarToRow (51) + : +- Scan parquet default.date_dim (50) + +- BroadcastExchange (56) + +- * Project (55) + +- * Filter (54) + +- * ColumnarToRow (53) + +- Scan parquet default.date_dim (52) + + +(47) Scan parquet default.date_dim +Output [2]: [d_date_sk#8, d_date#45] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct + +(48) ColumnarToRow [codegen id : 3] +Input [2]: [d_date_sk#8, d_date#45] -(46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +(49) Filter [codegen id : 3] +Input [2]: [d_date_sk#8, d_date#45] +Condition : isnotnull(d_date_sk#8) + +(50) Scan parquet default.date_dim +Output [2]: [d_date#46, d_week_seq#47] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#37), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(wr_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +ReadSchema: struct + +(51) ColumnarToRow [codegen id : 2] +Input [2]: [d_date#46, d_week_seq#47] + +(52) Scan parquet default.date_dim +Output [2]: [d_date#48, d_week_seq#49] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +ReadSchema: struct + +(53) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#48, d_week_seq#49] -(47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +(54) Filter [codegen id : 1] +Input [2]: [d_date#48, d_week_seq#49] +Condition : cast(d_date#48 as string) IN (2000-06-30,2000-09-27,2000-11-17) -(48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] -Condition : isnotnull(wr_item_sk#35) +(55) Project [codegen id : 1] +Output [1]: [d_week_seq#49] +Input [2]: [d_date#48, d_week_seq#49] -(49) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#38, i_item_id#39] +(56) BroadcastExchange +Input [1]: [d_week_seq#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] -(50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#35] -Right keys [1]: [i_item_sk#38] +(57) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [d_week_seq#47] +Right keys [1]: [d_week_seq#49] Join condition: None -(51) Project [codegen id : 16] -Output [3]: [wr_return_quantity#36, wr_returned_date_sk#37, i_item_id#39] -Input [5]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37, i_item_sk#38, i_item_id#39] +(58) Project [codegen id : 2] +Output [1]: [d_date#46] +Input [2]: [d_date#46, d_week_seq#47] -(52) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#40] +(59) BroadcastExchange +Input [1]: [d_date#46] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#51] -(53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#37] -Right keys [1]: [d_date_sk#40] +(60) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date#45] +Right keys [1]: [d_date#46] Join condition: None -(54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#36, i_item_id#39] -Input [4]: [wr_return_quantity#36, wr_returned_date_sk#37, i_item_id#39, d_date_sk#40] - -(55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#36, i_item_id#39] -Keys [1]: [i_item_id#39] -Functions [1]: [partial_sum(wr_return_quantity#36)] -Aggregate Attributes [1]: [sum#41] -Results [2]: [i_item_id#39, sum#42] - -(56) Exchange -Input [2]: [i_item_id#39, sum#42] -Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, [id=#43] - -(57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#39, sum#42] -Keys [1]: [i_item_id#39] -Functions [1]: [sum(wr_return_quantity#36)] -Aggregate Attributes [1]: [sum(wr_return_quantity#36)#44] -Results [2]: [i_item_id#39 AS item_id#45, sum(wr_return_quantity#36)#44 AS wr_item_qty#46] - -(58) BroadcastExchange -Input [2]: [item_id#45, wr_item_qty#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] - -(59) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#45] -Join condition: None +(61) Project [codegen id : 3] +Output [1]: [d_date_sk#8] +Input [2]: [d_date_sk#8, d_date#45] + +(62) BroadcastExchange +Input [1]: [d_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] + +Subquery:2 Hosting operator id = 16 Hosting Expression = cr_returned_date_sk#17 IN dynamicpruning#4 -(60) Project [codegen id : 18] -Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS sr_dev#48, cr_item_qty#33, (((cast(cr_item_qty#33 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS cr_dev#49, wr_item_qty#46, (((cast(wr_item_qty#46 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS wr_dev#50, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#51] -Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#33, item_id#45, wr_item_qty#46] +Subquery:3 Hosting operator id = 31 Hosting Expression = wr_returned_date_sk#30 IN dynamicpruning#4 -(61) TakeOrderedAndProject -Input [8]: [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] -Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index 7fc930f2885fd..29ff19d7450c8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -17,38 +17,40 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (3) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + Filter [d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [d_date] + BroadcastHashJoin [d_week_seq,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter - BroadcastExchange #2 + BroadcastExchange #5 WholeStageCodegen (1) Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (4) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [d_date] - BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Project [d_week_seq] - Filter [d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #6 WholeStageCodegen (11) @@ -65,10 +67,11 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 + ReusedExchange [i_item_sk,i_item_id] #5 InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #8 WholeStageCodegen (17) @@ -85,7 +88,8 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 + ReusedExchange [i_item_sk,i_item_id] #5 InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #2 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt index 84342b5469473..8e15a8b4c1e2a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt @@ -350,94 +350,94 @@ Right keys [1]: [customer_id#72] Join condition: None (60) Project [codegen id : 26] -Output [7]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#73, CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#75] +Output [8]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, year_total#73] Input [9]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, customer_id#72, year_total#73] (61) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_sold_date_sk#78 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] (63) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_customer_sk#76) +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] +Condition : isnotnull(ws_bill_customer_sk#75) (64) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#80, d_year#81] +Output [2]: [d_date_sk#79, d_year#80] (65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#79] -Right keys [1]: [d_date_sk#80] +Left keys [1]: [ws_sold_date_sk#78] +Right keys [1]: [d_date_sk#79] Join condition: None (66) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#80, d_year#81] +Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Input [6]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78, d_date_sk#79, d_year#80] (67) Exchange -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] -Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Arguments: hashpartitioning(ws_bill_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] (68) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] -Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Arguments: [ws_bill_customer_sk#75 ASC NULLS FIRST], false, 0 (69) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#83, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90] +Output [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] (70) Sort [codegen id : 31] -Input [8]: [c_customer_sk#83, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90] -Arguments: [c_customer_sk#83 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] +Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#76] -Right keys [1]: [c_customer_sk#83] +Left keys [1]: [ws_bill_customer_sk#75] +Right keys [1]: [c_customer_sk#82] Join condition: None (72) Project [codegen id : 32] -Output [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] -Input [12]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81, c_customer_sk#83, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90] +Output [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Input [12]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80, c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] (73) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#81] -Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#91] -Results [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, sum#92] +Input [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#90] +Results [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] (74) Exchange -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, sum#92] -Arguments: hashpartitioning(c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Arguments: hashpartitioning(c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, 5), ENSURE_REQUIREMENTS, [id=#92] (75) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81, sum#92] -Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#88, c_login#89, c_email_address#90, d_year#81] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))#94] -Results [2]: [c_customer_id#84 AS customer_id#95, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))#94,18,2) AS year_total#96] +Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93] +Results [2]: [c_customer_id#83 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] (76) Exchange -Input [2]: [customer_id#95, year_total#96] -Arguments: hashpartitioning(customer_id#95, 5), ENSURE_REQUIREMENTS, [id=#97] +Input [2]: [customer_id#94, year_total#95] +Arguments: hashpartitioning(customer_id#94, 5), ENSURE_REQUIREMENTS, [id=#96] (77) Sort [codegen id : 34] -Input [2]: [customer_id#95, year_total#96] -Arguments: [customer_id#95 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#94, year_total#95] +Arguments: [customer_id#94 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#95] -Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#96) / promote_precision(year_total#73)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#75) +Right keys [1]: [customer_id#94] +Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END) (79) Project [codegen id : 35] Output [4]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48] -Input [9]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#73, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END#75, customer_id#95, year_total#96] +Input [10]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, year_total#73, customer_id#94, year_total#95] (80) TakeOrderedAndProject Input [4]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48] @@ -468,7 +468,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2001)) AND isnotnull(d_date_sk (84) BroadcastExchange Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#97] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (88) @@ -493,10 +493,10 @@ Condition : ((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND isnotnull(d_date_ (88) BroadcastExchange Input [2]: [d_date_sk#30, d_year#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#98] Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#29 +Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#78 IN dynamicpruning#29 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt index 35e8fb614399f..b3f8a57ba0f5b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] WholeStageCodegen (35) Project [customer_id,customer_first_name,customer_last_name,customer_email_address] - SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE 0E-20BD END] + SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (26) - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) 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 db497ae40b481..1e64471e096cb 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 @@ -223,61 +223,61 @@ Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_yea Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (rn#39 + 1) AS (v1_lag.rn + 1)#40] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] (40) Exchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] (41) Sort [codegen id : 23] -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (v1_lag.rn + 1)#40 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (v1_lag.rn + 1)#40] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None (43) Project [codegen id : 24] Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37] -Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, (v1_lag.rn + 1)#40] +Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] (44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Output [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] (45) Sort [codegen id : 33] -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST], false, 0 (46) Window -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] +Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Arguments: [rank(d_year#45, d_moy#46) windowspecdefinition(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#48], [i_category#41, i_brand#42, s_store_name#43, s_company_name#44], [d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (rn#49 - 1) AS (v1_lead.rn - 1)#50] -Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] +Output [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Input [8]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47, rn#48] (48) Exchange -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] -Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Arguments: hashpartitioning(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1), 5), ENSURE_REQUIREMENTS, [id=#49] (49) Sort [codegen id : 35] -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (v1_lead.rn - 1)#50 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, (rn#48 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (v1_lead.rn - 1)#50] +Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None (51) Project [codegen id : 36] -Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#52, sum_sales#48 AS nsum#53] -Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, (v1_lead.rn - 1)#50] +Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] +Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] (52) TakeOrderedAndProject -Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#52, nsum#53] +Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt index f522534d62c80..5f64a22717270 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] WholeStageCodegen (36) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,s_store_name,s_company_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lag.rn + 1)] #8 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #8 WholeStageCodegen (22) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,(v1_lead.rn - 1)] #10 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #10 WholeStageCodegen (34) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter 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 0d87664dfb7a5..e7cd713d07cb4 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 @@ -223,61 +223,61 @@ Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sale Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (rn#37 + 1) AS (v1_lag.rn + 1)#38] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] (40) Exchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#38] (41) Sort [codegen id : 23] -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (v1_lag.rn + 1)#38 ASC NULLS FIRST], false, 0 +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (v1_lag.rn + 1)#38] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None (43) Project [codegen id : 24] Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, (v1_lag.rn + 1)#38] +Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] (44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] (45) Sort [codegen id : 33] -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 (46) Window -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] +Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#39, i_brand#40, cc_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#45], [i_category#39, i_brand#40, cc_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (rn#46 - 1) AS (v1_lead.rn - 1)#47] -Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] +Output [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] (48) Exchange -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] -Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: hashpartitioning(i_category#39, i_brand#40, cc_name#41, (rn#45 - 1), 5), ENSURE_REQUIREMENTS, [id=#46] (49) Sort [codegen id : 35] -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (v1_lead.rn - 1)#47 ASC NULLS FIRST], false, 0 +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, (rn#45 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (v1_lead.rn - 1)#47] +Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None (51) Project [codegen id : 36] -Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#49, sum_sales#45 AS nsum#50] -Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#40, i_brand#41, cc_name#42, sum_sales#45, (v1_lead.rn - 1)#47] +Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] +Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] (52) TakeOrderedAndProject -Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#49, nsum#50] +Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt index f5acc760c88f3..b464f558bbc1a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] WholeStageCodegen (36) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lead.rn - 1)] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter WholeStageCodegen (24) Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,(v1_lag.rn + 1)] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter WholeStageCodegen (12) Sort [i_category,i_brand,cc_name,rn] @@ -74,9 +74,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (23) - Sort [i_category,i_brand,cc_name,(v1_lag.rn + 1)] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [i_category,i_brand,cc_name,(v1_lag.rn + 1)] #8 + Exchange [i_category,i_brand,cc_name,rn] #8 WholeStageCodegen (22) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter @@ -91,9 +91,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3 InputAdapter WholeStageCodegen (35) - Sort [i_category,i_brand,cc_name,(v1_lead.rn - 1)] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [i_category,i_brand,cc_name,(v1_lead.rn - 1)] #10 + Exchange [i_category,i_brand,cc_name,rn] #10 WholeStageCodegen (34) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt index 67d964ee31184..fea0bcbbef17e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt @@ -350,94 +350,94 @@ Right keys [1]: [customer_id#56] Join condition: None (60) Project [codegen id : 26] -Output [6]: [customer_id#17, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#57, CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#38) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END AS CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#59] +Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] (61) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +Output [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#23)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] (63) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Condition : isnotnull(ws_bill_customer_sk#60) +Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] +Condition : isnotnull(ws_bill_customer_sk#59) (64) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#63, d_year#64] +Output [2]: [d_date_sk#62, d_year#63] (65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#62] -Right keys [1]: [d_date_sk#63] +Left keys [1]: [ws_sold_date_sk#61] +Right keys [1]: [d_date_sk#62] Join condition: None (66) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64] -Input [5]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62, d_date_sk#63, d_year#64] +Output [3]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63] +Input [5]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61, d_date_sk#62, d_year#63] (67) Exchange -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64] -Arguments: hashpartitioning(ws_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63] +Arguments: hashpartitioning(ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#64] (68) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64] -Arguments: [ws_bill_customer_sk#60 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63] +Arguments: [ws_bill_customer_sk#59 ASC NULLS FIRST], false, 0 (69) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69] +Output [4]: [c_customer_sk#65, c_customer_id#66, c_first_name#67, c_last_name#68] (70) Sort [codegen id : 31] -Input [4]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69] -Arguments: [c_customer_sk#66 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#65, c_customer_id#66, c_first_name#67, c_last_name#68] +Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#60] -Right keys [1]: [c_customer_sk#66] +Left keys [1]: [ws_bill_customer_sk#59] +Right keys [1]: [c_customer_sk#65] Join condition: None (72) Project [codegen id : 32] -Output [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, ws_net_paid#61, d_year#64] -Input [7]: [ws_bill_customer_sk#60, ws_net_paid#61, d_year#64, c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69] +Output [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, ws_net_paid#60, d_year#63] +Input [7]: [ws_bill_customer_sk#59, ws_net_paid#60, d_year#63, c_customer_sk#65, c_customer_id#66, c_first_name#67, c_last_name#68] (73) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, ws_net_paid#61, d_year#64] -Keys [4]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#61))] -Aggregate Attributes [1]: [sum#70] -Results [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, sum#71] +Input [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, ws_net_paid#60, d_year#63] +Keys [4]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#60))] +Aggregate Attributes [1]: [sum#69] +Results [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, sum#70] (74) Exchange -Input [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, sum#71] -Arguments: hashpartitioning(c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, sum#70] +Arguments: hashpartitioning(c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, 5), ENSURE_REQUIREMENTS, [id=#71] (75) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64, sum#71] -Keys [4]: [c_customer_id#67, c_first_name#68, c_last_name#69, d_year#64] -Functions [1]: [sum(UnscaledValue(ws_net_paid#61))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#61))#73] -Results [2]: [c_customer_id#67 AS customer_id#74, MakeDecimal(sum(UnscaledValue(ws_net_paid#61))#73,17,2) AS year_total#75] +Input [5]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63, sum#70] +Keys [4]: [c_customer_id#66, c_first_name#67, c_last_name#68, d_year#63] +Functions [1]: [sum(UnscaledValue(ws_net_paid#60))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#60))#72] +Results [2]: [c_customer_id#66 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#60))#72,17,2) AS year_total#74] (76) Exchange -Input [2]: [customer_id#74, year_total#75] -Arguments: hashpartitioning(customer_id#74, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [2]: [customer_id#73, year_total#74] +Arguments: hashpartitioning(customer_id#73, 5), ENSURE_REQUIREMENTS, [id=#75] (77) Sort [codegen id : 34] -Input [2]: [customer_id#74, year_total#75] -Arguments: [customer_id#74 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#73, year_total#74] +Arguments: [customer_id#73 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#74] -Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#75) / promote_precision(year_total#57)), DecimalType(37,20), true) ELSE null END > CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#59) +Right keys [1]: [customer_id#73] +Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#38) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) (79) Project [codegen id : 35] Output [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] -Input [8]: [customer_id#17, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#57, CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END#59, customer_id#74, year_total#75] +Input [9]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57, customer_id#73, year_total#74] (80) TakeOrderedAndProject Input [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] @@ -468,7 +468,7 @@ Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2001)) AND d_year#6 IN (2001, (84) BroadcastExchange Input [2]: [d_date_sk#5, d_year#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (88) @@ -493,10 +493,10 @@ Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (20 (88) BroadcastExchange Input [2]: [d_date_sk#24, d_year#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#23 +Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#23 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt index e4c930b0c6f25..99e72fe265b6b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (35) Project [customer_id,customer_first_name,customer_last_name] - SortMergeJoin [customer_id,customer_id,year_total,year_total,CASE WHEN (t_s_firstyear.year_total > 0.00BD) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST(NULL AS DECIMAL(37,20)) END] + SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (26) - Project [customer_id,customer_id,customer_first_name,customer_last_name,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) From 29507cc9c1290969cebc0df6ecf526290f4bf9c9 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 9 Aug 2021 20:05:52 +0800 Subject: [PATCH 09/17] fix --- .../optimizer/PullOutJoinConditionSuite.scala | 10 ---------- .../test/scala/org/apache/spark/sql/JoinSuite.scala | 2 +- .../apache/spark/sql/execution/PlannerSuite.scala | 13 +++++-------- 3 files changed, 6 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala index e1a53d5c72388..8be7fe666be1c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala @@ -70,16 +70,6 @@ class PullOutJoinConditionSuite extends PlanTest { } } - test("Negative case: Non broadcast hash join") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000000") { - val joinType = Inner - val udf = Upper("y.d".attr) - val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) - - comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) - } - } - test("Negative case: all children are Attributes") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val condition = Option("x.a".attr === "y.d".attr) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index abfc19ac6dffd..cddddefb91bba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -1057,7 +1057,7 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val pythonEvals = collect(joinNode.get) { case p: BatchEvalPythonExec => p } - assert(pythonEvals.size == 2) + assert(pythonEvals.size == 4) checkAnswer(df, Row(1, 2, 1, 2) :: Nil) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 5b44e26e438d6..1fbd86d3ae5ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -994,12 +994,9 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { |WHERE t1id * 10 = t3.id3 * 10 """.stripMargin).queryExecution.executedPlan val sortNodes = collect(planned) { case s: SortExec => s } - assert(sortNodes.size == 4) + assert(sortNodes.size == 3) val exchangeNodes = collect(planned) { case e: ShuffleExchangeExec => e } - assert(exchangeNodes.size == 4) - - val projects = collect(planned) { case p: ProjectExec => p } - assert(!projects.exists(_.outputPartitioning.isInstanceOf[HashPartitioning])) + assert(exchangeNodes.size == 3) } } } @@ -1098,13 +1095,13 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { // t12 is already sorted on `t1.id * 2`. and we need to sort it on `2 * t12.id` // for 2nd join. So sorting on t12 can be avoided val sortNodes = planned.collect { case s: SortExec => s } - assert(sortNodes.size == 4) + assert(sortNodes.size == 3) val outputOrdering = planned.outputOrdering assert(outputOrdering.size == 1) // Sort order should have 3 childrens, not 4. This is because t1.id*2 and 2*t1.id are same - assert(outputOrdering.head.children.size == 2) + assert(outputOrdering.head.children.size == 3) assert(outputOrdering.head.children.count(_.isInstanceOf[AttributeReference]) == 2) - assert(outputOrdering.head.children.count(_.isInstanceOf[Multiply]) == 0) + assert(outputOrdering.head.children.count(_.isInstanceOf[Multiply]) == 1) } } } From 07eda27de9622d97f807a7891af824e203e6b8ff Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 9 Aug 2021 21:02:10 +0800 Subject: [PATCH 10/17] fix --- .../apache/spark/sql/execution/WholeStageCodegenSuite.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 6cc6e33dd688a..80dc6edd0300d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -145,7 +145,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val df = spark.range(10).join(broadcast(smallDF), col("k") === col("id")) assert(df.queryExecution.executedPlan.find(p => p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[BroadcastHashJoinExec]).isDefined) + p.asInstanceOf[WholeStageCodegenExec].child + .children.head.isInstanceOf[BroadcastHashJoinExec]).isDefined) assert(df.collect() === Array(Row(1, 1, "1"), Row(1, 1, "1"), Row(2, 2, "2"))) } @@ -317,7 +318,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .join(df3, $"k1" <= $"k3", "left_outer") hasJoinInCodegen = twoJoinsDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(BroadcastNestedLoopJoinExec( - _: BroadcastNestedLoopJoinExec, _, _, _, _)) => true + ProjectExec(_, _: BroadcastNestedLoopJoinExec), _, _, _, _)) => true }.size === 1 assert(hasJoinInCodegen == codegenEnabled) checkAnswer(twoJoinsDF, From 58c6c60c08faddc5fd78b5d83d0e881e7fadbec7 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 2 Nov 2021 20:49:36 +0800 Subject: [PATCH 11/17] Merge Upstream --- .../sql/catalyst/optimizer/PullOutJoinCondition.scala | 10 +++++----- .../org/apache/spark/sql/execution/PlannerSuite.scala | 8 ++++++++ .../spark/sql/execution/WholeStageCodegenSuite.scala | 8 ++++---- 3 files changed, 17 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala index b5384c653a657..e957ccf9923e5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala @@ -32,8 +32,8 @@ import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN * {{{ * SELECT * FROM t1 JOIN t2 ON t1.a + 10 = t2.x ==> * Project [a#0, b#1, x#2, y#3] - * +- Join Inner, ((spark_catalog.default.t1.a + 10)#8 = x#2) - * :- Project [a#0, b#1, (a#0 + 10) AS (spark_catalog.default.t1.a + 10)#8] + * +- Join Inner, ((a#0 + 10)#8 = x#2) + * :- Project [a#0, b#1, (a#0 + 10) AS (a#0 + 10)#8] * : +- Filter isnotnull((a#0 + 10)) * : +- Relation default.t1[a#0,b#1] parquet * +- Filter isnotnull(x#2) @@ -44,7 +44,7 @@ object PullOutJoinCondition extends Rule[LogicalPlan] with JoinSelectionHelper with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(_.containsPattern(JOIN)) { - case j @ Join(left, right, _, Some(condition), _) if j.resolved && !j.isStreaming => + case j @ Join(left, right, _, Some(condition), _) if j.resolved => val complexExpressions = splitConjunctivePredicates(condition).flatMap(_.children).flatMap { case e: Expression if !e.foldable && e.children.nonEmpty => Seq(e) case _ => Nil @@ -54,9 +54,9 @@ object PullOutJoinCondition extends Rule[LogicalPlan] val rightComplexExpressions = complexExpressions.filter(canEvaluate(_, right)) val leftComplexExpressionMap = - leftComplexExpressions.map(e => e.canonicalized -> Alias(e, e.sql)()).toMap + leftComplexExpressions.map(e => e.canonicalized -> Alias(e, e.toString)()).toMap val rightComplexExpressionMap = - rightComplexExpressions.map(e => e.canonicalized -> Alias(e, e.sql)()).toMap + rightComplexExpressions.map(e => e.canonicalized -> Alias(e, e.toString)()).toMap val allComplexExpressionMap = leftComplexExpressionMap ++ rightComplexExpressionMap if (allComplexExpressionMap.nonEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 460805228f89e..df310cbaee455 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -999,6 +999,14 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { assert(sortNodes.size == 3) val exchangeNodes = collect(planned) { case e: ShuffleExchangeExec => e } assert(exchangeNodes.size == 3) + + val projects = collect(planned) { case p: ProjectExec => p } + assert(projects.exists(_.outputPartitioning match { + case HashPartitioning(Seq(Multiply(ar1: AttributeReference, _, _)), _) => + ar1.name == "t1id" + case _ => + false + })) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 80dc6edd0300d..d8cee2320c364 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -143,10 +143,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val schema = new StructType().add("k", IntegerType).add("v", StringType) val smallDF = spark.createDataFrame(rdd, schema) val df = spark.range(10).join(broadcast(smallDF), col("k") === col("id")) - assert(df.queryExecution.executedPlan.find(p => - p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].child - .children.head.isInstanceOf[BroadcastHashJoinExec]).isDefined) + val broadcastHashJoin = df.queryExecution.executedPlan.find { + case WholeStageCodegenExec(ProjectExec(_, _: BroadcastHashJoinExec)) => true + } + assert(broadcastHashJoin.isDefined) assert(df.collect() === Array(Row(1, 1, "1"), Row(1, 1, "1"), Row(2, 2, "2"))) } From 0e0c085932c7232d9a5d8f108c124ed6380e6d3e Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 2 Nov 2021 21:21:45 +0800 Subject: [PATCH 12/17] Add more tests --- .../optimizer/PullOutJoinCondition.scala | 11 +++++---- .../optimizer/PullOutJoinConditionSuite.scala | 24 +++++++++++++++++++ .../org/apache/spark/sql/JoinSuite.scala | 2 +- .../execution/WholeStageCodegenSuite.scala | 9 ++++--- 4 files changed, 35 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala index e957ccf9923e5..5543e87da3ce2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala @@ -32,8 +32,8 @@ import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN * {{{ * SELECT * FROM t1 JOIN t2 ON t1.a + 10 = t2.x ==> * Project [a#0, b#1, x#2, y#3] - * +- Join Inner, ((a#0 + 10)#8 = x#2) - * :- Project [a#0, b#1, (a#0 + 10) AS (a#0 + 10)#8] + * +- Join Inner, ((spark_catalog.default.t1.a + 10)#8 = x#2) + * :- Project [a#0, b#1, (a#0 + 10) AS (spark_catalog.default.t1.a + 10)#8] * : +- Filter isnotnull((a#0 + 10)) * : +- Relation default.t1[a#0,b#1] parquet * +- Filter isnotnull(x#2) @@ -44,7 +44,8 @@ object PullOutJoinCondition extends Rule[LogicalPlan] with JoinSelectionHelper with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(_.containsPattern(JOIN)) { - case j @ Join(left, right, _, Some(condition), _) if j.resolved => + case j @ Join(left, right, _, Some(condition), _) + if j.resolved && !canPlanAsBroadcastHashJoin(j, conf) => val complexExpressions = splitConjunctivePredicates(condition).flatMap(_.children).flatMap { case e: Expression if !e.foldable && e.children.nonEmpty => Seq(e) case _ => Nil @@ -54,9 +55,9 @@ object PullOutJoinCondition extends Rule[LogicalPlan] val rightComplexExpressions = complexExpressions.filter(canEvaluate(_, right)) val leftComplexExpressionMap = - leftComplexExpressions.map(e => e.canonicalized -> Alias(e, e.toString)()).toMap + leftComplexExpressions.map(e => e.canonicalized -> Alias(e, e.sql)()).toMap val rightComplexExpressionMap = - rightComplexExpressions.map(e => e.canonicalized -> Alias(e, e.toString)()).toMap + rightComplexExpressions.map(e => e.canonicalized -> Alias(e, e.sql)()).toMap val allComplexExpressionMap = leftComplexExpressionMap ++ rightComplexExpressionMap if (allComplexExpressionMap.nonEmpty) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala index 8be7fe666be1c..af22ad7f2cb1d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala @@ -70,6 +70,22 @@ class PullOutJoinConditionSuite extends PlanTest { } } + test("Push down non-equality join condition") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val joinType = Inner + val udf = "x.b".attr + 1 + val originalQuery = x.join(y, joinType, Option(udf > "y.e".attr)) + .select("x.a".attr, "y.e".attr) + val correctAnswer = + x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join( + y.select("y.d".attr, "y.e".attr), + joinType, Option(s"`${udf.sql}`".attr > "y.e".attr)) + .select("x.a".attr, "y.e".attr) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + } + test("Negative case: all children are Attributes") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val condition = Option("x.a".attr === "y.d".attr) @@ -89,4 +105,12 @@ class PullOutJoinConditionSuite extends PlanTest { comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } } + + test("Negative case: BroadcastHashJoin") { + Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => + val originalQuery = x.join(y, Inner, Option("x.a".attr === udf)) + .select("x.a".attr, "y.e".attr) + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 10b49a6efc76d..77493afe43145 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -1057,7 +1057,7 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val pythonEvals = collect(joinNode.get) { case p: BatchEvalPythonExec => p } - assert(pythonEvals.size == 4) + assert(pythonEvals.size == 2) checkAnswer(df, Row(1, 2, 1, 2) :: Nil) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index d8cee2320c364..6cc6e33dd688a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -143,10 +143,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val schema = new StructType().add("k", IntegerType).add("v", StringType) val smallDF = spark.createDataFrame(rdd, schema) val df = spark.range(10).join(broadcast(smallDF), col("k") === col("id")) - val broadcastHashJoin = df.queryExecution.executedPlan.find { - case WholeStageCodegenExec(ProjectExec(_, _: BroadcastHashJoinExec)) => true - } - assert(broadcastHashJoin.isDefined) + assert(df.queryExecution.executedPlan.find(p => + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[BroadcastHashJoinExec]).isDefined) assert(df.collect() === Array(Row(1, 1, "1"), Row(1, 1, "1"), Row(2, 2, "2"))) } @@ -318,7 +317,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .join(df3, $"k1" <= $"k3", "left_outer") hasJoinInCodegen = twoJoinsDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(BroadcastNestedLoopJoinExec( - ProjectExec(_, _: BroadcastNestedLoopJoinExec), _, _, _, _)) => true + _: BroadcastNestedLoopJoinExec, _, _, _, _)) => true }.size === 1 assert(hasJoinInCodegen == codegenEnabled) checkAnswer(twoJoinsDF, From c63f784b7ed6a5cc461c64972221be980d197568 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 3 Nov 2021 15:25:01 +0800 Subject: [PATCH 13/17] Push down EqualNullSafe join condition --- .../optimizer/PullOutJoinCondition.scala | 10 ++++----- .../optimizer/PullOutJoinConditionSuite.scala | 22 ++++++++++++++++--- 2 files changed, 24 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala index 5543e87da3ce2..8370b5cfa0e73 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala @@ -17,16 +17,16 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, PredicateHelper} +import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, Predicate, PredicateHelper} import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN /** - * This rule ensures that [[Join]] condition doesn't contain complex expressions in the + * This rule ensures that [[Join]] keys doesn't contain complex expressions in the * optimization phase. * - * Complex condition expressions are pulled out to a [[Project]] node under [[Join]] and are + * Complex expressions are pulled out to a [[Project]] node under [[Join]] and are * referenced in join condition. * * {{{ @@ -46,8 +46,8 @@ object PullOutJoinCondition extends Rule[LogicalPlan] def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(_.containsPattern(JOIN)) { case j @ Join(left, right, _, Some(condition), _) if j.resolved && !canPlanAsBroadcastHashJoin(j, conf) => - val complexExpressions = splitConjunctivePredicates(condition).flatMap(_.children).flatMap { - case e: Expression if !e.foldable && e.children.nonEmpty => Seq(e) + val complexExpressions = splitConjunctivePredicates(condition).flatMap { + case p: Predicate => p.children.filter(e => !e.foldable && e.children.nonEmpty) case _ => Nil } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala index af22ad7f2cb1d..738e37bed03b1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala @@ -39,7 +39,7 @@ class PullOutJoinConditionSuite extends PlanTest { private val x = testRelation.subquery('x) private val y = testRelation1.subquery('y) - test("Push down join condition evaluation(String expressions)") { + test("Push down join keys evaluation(String expressions)") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val joinType = Inner Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => @@ -54,7 +54,7 @@ class PullOutJoinConditionSuite extends PlanTest { } } - test("Push down join condition evaluation(null expressions)") { + test("Push down join keys evaluation(null expressions)") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val joinType = Inner val udf = Coalesce(Seq("x.b".attr, "x.c".attr)) @@ -70,7 +70,23 @@ class PullOutJoinConditionSuite extends PlanTest { } } - test("Push down non-equality join condition") { + test("Push down EqualNullSafe join condition") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val joinType = Inner + val udf = "x.b".attr + 1 + val originalQuery = x.join(y, joinType, Option(udf <=> "y.e".attr)) + .select("x.a".attr, "y.e".attr) + val correctAnswer = + x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join( + y.select("y.d".attr, "y.e".attr), + joinType, Option(s"`${udf.sql}`".attr <=> "y.e".attr)) + .select("x.a".attr, "y.e".attr) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + } + + test("Push down non-equality join keys") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val joinType = Inner val udf = "x.b".attr + 1 From 62a5ac08204a2f7b16a7828f70bb6d1fa351984b Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 4 Nov 2021 17:48:09 +0800 Subject: [PATCH 14/17] Address comments --- .../optimizer/PullOutJoinCondition.scala | 60 +++++++++++-------- .../optimizer/PullOutJoinConditionSuite.scala | 58 +++++++++++------- 2 files changed, 73 insertions(+), 45 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala index 8370b5cfa0e73..232c57d941d1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala @@ -17,8 +17,11 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, Predicate, PredicateHelper} -import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project} +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.catalyst.expressions.{Alias, And, EqualTo, NamedExpression, PredicateHelper} +import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN @@ -32,8 +35,8 @@ import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN * {{{ * SELECT * FROM t1 JOIN t2 ON t1.a + 10 = t2.x ==> * Project [a#0, b#1, x#2, y#3] - * +- Join Inner, ((spark_catalog.default.t1.a + 10)#8 = x#2) - * :- Project [a#0, b#1, (a#0 + 10) AS (spark_catalog.default.t1.a + 10)#8] + * +- Join Inner, (_left_complex_expr0#8 = x#2) + * :- Project [a#0, b#1, (a#0 + 10) AS _left_complex_expr0#8] * : +- Filter isnotnull((a#0 + 10)) * : +- Relation default.t1[a#0,b#1] parquet * +- Filter isnotnull(x#2) @@ -44,31 +47,38 @@ object PullOutJoinCondition extends Rule[LogicalPlan] with JoinSelectionHelper with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(_.containsPattern(JOIN)) { - case j @ Join(left, right, _, Some(condition), _) + case j @ ExtractEquiJoinKeys(_, leftKeys, rightKeys, otherPredicates, _, left, right, _) if j.resolved && !canPlanAsBroadcastHashJoin(j, conf) => - val complexExpressions = splitConjunctivePredicates(condition).flatMap { - case p: Predicate => p.children.filter(e => !e.foldable && e.children.nonEmpty) - case _ => Nil - } + val complexLeftJoinKeys = new ArrayBuffer[NamedExpression]() + val complexRightJoinKeys = new ArrayBuffer[NamedExpression]() - val leftComplexExpressions = complexExpressions.filter(canEvaluate(_, left)) - val rightComplexExpressions = complexExpressions.filter(canEvaluate(_, right)) - - val leftComplexExpressionMap = - leftComplexExpressions.map(e => e.canonicalized -> Alias(e, e.sql)()).toMap - val rightComplexExpressionMap = - rightComplexExpressions.map(e => e.canonicalized -> Alias(e, e.sql)()).toMap - val allComplexExpressionMap = leftComplexExpressionMap ++ rightComplexExpressionMap + val newLeftJoinKeys = leftKeys.zipWithIndex.map { case (expr, index) => + if (!expr.foldable && expr.children.nonEmpty) { + val ne = Alias(expr, s"_left_complex_expr$index")() + complexLeftJoinKeys += ne + ne.toAttribute + } else { + expr + } + } - if (allComplexExpressionMap.nonEmpty) { - val newCondition = condition.transformDown { - case e: Expression - if e.children.nonEmpty && allComplexExpressionMap.contains(e.canonicalized) => - allComplexExpressionMap.get(e.canonicalized).map(_.toAttribute).getOrElse(e) + val newRightJoinKeys = rightKeys.zipWithIndex.map { case (expr, index) => + if (!expr.foldable && expr.children.nonEmpty) { + val ne = Alias(expr, s"_right_complex_expr$index")() + complexRightJoinKeys += ne + ne.toAttribute + } else { + expr } - val newLeft = Project(left.output ++ leftComplexExpressionMap.values, left) - val newRight = Project(right.output ++ rightComplexExpressionMap.values, right) - Project(j.output, j.copy(left = newLeft, right = newRight, condition = Some(newCondition))) + } + + if (complexLeftJoinKeys.nonEmpty || complexRightJoinKeys.nonEmpty) { + val newLeft = Project(left.output ++ complexLeftJoinKeys, left) + val newRight = Project(right.output ++ complexRightJoinKeys, right) + val newCond = (newLeftJoinKeys.zip(newRightJoinKeys) + .map { case (l, r) => EqualTo(l, r) } ++ otherPredicates) + .reduceLeftOption(And) + Project(j.output, j.copy(left = newLeft, right = newRight, condition = newCond)) } else { j } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala index 738e37bed03b1..643b749f791e4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{Alias, Coalesce, Substring, Upper} +import org.apache.spark.sql.catalyst.expressions.{Alias, Coalesce, IsNull, Literal, Substring, Upper} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -46,9 +46,9 @@ class PullOutJoinConditionSuite extends PlanTest { val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) .select("x.a".attr, "y.e".attr) val correctAnswer = x.select("x.a".attr, "x.b".attr, "x.c".attr) - .join(y.select("y.d".attr, "y.e".attr, Alias(udf, udf.sql)()), - joinType, Option("x.a".attr === s"`${udf.sql}`".attr)).select("x.a".attr, "y.e".attr) - + .join(y.select("y.d".attr, "y.e".attr, Alias(udf, "_right_complex_expr0")()), + joinType, + Option("x.a".attr === "_right_complex_expr0".attr)).select("x.a".attr, "y.e".attr) comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } } @@ -61,15 +61,31 @@ class PullOutJoinConditionSuite extends PlanTest { val originalQuery = x.join(y, joinType, Option(udf === "y.e".attr)) .select("x.a".attr, "y.e".attr) val correctAnswer = - x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join( + x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, "_left_complex_expr0")()).join( y.select("y.d".attr, "y.e".attr), - joinType, Option(s"`${udf.sql}`".attr === "y.e".attr)) + joinType, Option("_left_complex_expr0".attr === "y.e".attr)) .select("x.a".attr, "y.e".attr) comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } } + test("Join condition contains other predicates") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val udf = Substring("y.d".attr, 1, 5) + val joinType = Inner + val originalQuery = x.join(y, joinType, Option("x.a".attr === udf && "x.b".attr > "y.e".attr)) + .select("x.a".attr, "y.e".attr) + val correctAnswer = + x.select("x.a".attr, "x.b".attr, "x.c".attr).join( + y.select("y.d".attr, "y.e".attr, Alias(udf, "_right_complex_expr0")()), + joinType, Option("x.a".attr === "_right_complex_expr0".attr && "x.b".attr > "y.e".attr)) + .select("x.a".attr, "y.e".attr) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } + } + test("Push down EqualNullSafe join condition") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val joinType = Inner @@ -77,28 +93,28 @@ class PullOutJoinConditionSuite extends PlanTest { val originalQuery = x.join(y, joinType, Option(udf <=> "y.e".attr)) .select("x.a".attr, "y.e".attr) val correctAnswer = - x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join( - y.select("y.d".attr, "y.e".attr), - joinType, Option(s"`${udf.sql}`".attr <=> "y.e".attr)) + x.select("x.a".attr, "x.b".attr, "x.c".attr, + Alias(Coalesce(Seq(udf, Literal(0))), "_left_complex_expr0")(), + Alias(IsNull(udf), "_left_complex_expr1")()).join( + y.select("y.d".attr, "y.e".attr, + Alias(Coalesce(Seq("y.e".attr, Literal(0))), "_right_complex_expr0")(), + Alias(IsNull("y.e".attr), "_right_complex_expr1")()), + joinType, Option("_left_complex_expr0".attr === "_right_complex_expr0".attr && + "_left_complex_expr1".attr === "_right_complex_expr1".attr)) .select("x.a".attr, "y.e".attr) comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } } - test("Push down non-equality join keys") { + test("Negative case: non-equality join keys") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val joinType = Inner val udf = "x.b".attr + 1 val originalQuery = x.join(y, joinType, Option(udf > "y.e".attr)) .select("x.a".attr, "y.e".attr) - val correctAnswer = - x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join( - y.select("y.d".attr, "y.e".attr), - joinType, Option(s"`${udf.sql}`".attr > "y.e".attr)) - .select("x.a".attr, "y.e".attr) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) } } @@ -123,10 +139,12 @@ class PullOutJoinConditionSuite extends PlanTest { } test("Negative case: BroadcastHashJoin") { - Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => - val originalQuery = x.join(y, Inner, Option("x.a".attr === udf)) - .select("x.a".attr, "y.e".attr) - comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10MB") { + Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => + val originalQuery = x.join(y, Inner, Option("x.a".attr === udf)) + .select("x.a".attr, "y.e".attr) + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) + } } } } From b988b9e2025481774e1ec552dad717ac81dfd201 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 4 Nov 2021 22:57:02 +0800 Subject: [PATCH 15/17] Remove !canPlanAsBroadcastHashJoin(j, conf) --- .../optimizer/PullOutJoinCondition.scala | 19 ++- .../optimizer/PullOutJoinConditionSuite.scala | 127 ++++++++---------- .../org/apache/spark/sql/SQLQuerySuite.scala | 29 +++- 3 files changed, 91 insertions(+), 84 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala index 232c57d941d1c..2c0efb8bae3a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinCondition.scala @@ -35,26 +35,25 @@ import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN * {{{ * SELECT * FROM t1 JOIN t2 ON t1.a + 10 = t2.x ==> * Project [a#0, b#1, x#2, y#3] - * +- Join Inner, (_left_complex_expr0#8 = x#2) - * :- Project [a#0, b#1, (a#0 + 10) AS _left_complex_expr0#8] + * +- Join Inner, ((spark_catalog.default.t1.a + 10)#8 = x#2) + * :- Project [a#0, b#1, (a#0 + 10) AS (spark_catalog.default.t1.a + 10)#8] * : +- Filter isnotnull((a#0 + 10)) * : +- Relation default.t1[a#0,b#1] parquet * +- Filter isnotnull(x#2) * +- Relation default.t2[x#2,y#3] parquet * }}} */ -object PullOutJoinCondition extends Rule[LogicalPlan] - with JoinSelectionHelper with PredicateHelper { +object PullOutJoinCondition extends Rule[LogicalPlan] with PredicateHelper { - def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(_.containsPattern(JOIN)) { + def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithPruning(_.containsPattern(JOIN)) { case j @ ExtractEquiJoinKeys(_, leftKeys, rightKeys, otherPredicates, _, left, right, _) - if j.resolved && !canPlanAsBroadcastHashJoin(j, conf) => + if j.resolved => val complexLeftJoinKeys = new ArrayBuffer[NamedExpression]() val complexRightJoinKeys = new ArrayBuffer[NamedExpression]() - val newLeftJoinKeys = leftKeys.zipWithIndex.map { case (expr, index) => + val newLeftJoinKeys = leftKeys.map { expr => if (!expr.foldable && expr.children.nonEmpty) { - val ne = Alias(expr, s"_left_complex_expr$index")() + val ne = Alias(expr, expr.sql)() complexLeftJoinKeys += ne ne.toAttribute } else { @@ -62,9 +61,9 @@ object PullOutJoinCondition extends Rule[LogicalPlan] } } - val newRightJoinKeys = rightKeys.zipWithIndex.map { case (expr, index) => + val newRightJoinKeys = rightKeys.map { expr => if (!expr.foldable && expr.children.nonEmpty) { - val ne = Alias(expr, s"_right_complex_expr$index")() + val ne = Alias(expr, expr.sql)() complexRightJoinKeys += ne ne.toAttribute } else { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala index 643b749f791e4..eeb85b905d313 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala @@ -39,68 +39,67 @@ class PullOutJoinConditionSuite extends PlanTest { private val x = testRelation.subquery('x) private val y = testRelation1.subquery('y) - test("Push down join keys evaluation(String expressions)") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val joinType = Inner - Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => - val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) - .select("x.a".attr, "y.e".attr) - val correctAnswer = x.select("x.a".attr, "x.b".attr, "x.c".attr) - .join(y.select("y.d".attr, "y.e".attr, Alias(udf, "_right_complex_expr0")()), - joinType, - Option("x.a".attr === "_right_complex_expr0".attr)).select("x.a".attr, "y.e".attr) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) - } + test("Pull out join keys evaluation(String expressions)") { + val joinType = Inner + Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => + val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) + .select("x.a".attr, "y.e".attr) + val correctAnswer = x.select("x.a".attr, "x.b".attr, "x.c".attr) + .join(y.select("y.d".attr, "y.e".attr, Alias(udf, udf.sql)()), + joinType, Option("x.a".attr === s"`${udf.sql}`".attr)).select("x.a".attr, "y.e".attr) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } } - test("Push down join keys evaluation(null expressions)") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val joinType = Inner - val udf = Coalesce(Seq("x.b".attr, "x.c".attr)) - val originalQuery = x.join(y, joinType, Option(udf === "y.e".attr)) + test("Pull out join keys evaluation(null expressions)") { + val joinType = Inner + val udf = Coalesce(Seq("x.b".attr, "x.c".attr)) + val originalQuery = x.join(y, joinType, Option(udf === "y.e".attr)) + .select("x.a".attr, "y.e".attr) + val correctAnswer = + x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join( + y.select("y.d".attr, "y.e".attr), + joinType, Option(s"`${udf.sql}`".attr === "y.e".attr)) .select("x.a".attr, "y.e".attr) - val correctAnswer = - x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, "_left_complex_expr0")()).join( - y.select("y.d".attr, "y.e".attr), - joinType, Option("_left_complex_expr0".attr === "y.e".attr)) - .select("x.a".attr, "y.e".attr) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) - } + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } - test("Join condition contains other predicates") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val udf = Substring("y.d".attr, 1, 5) - val joinType = Inner - val originalQuery = x.join(y, joinType, Option("x.a".attr === udf && "x.b".attr > "y.e".attr)) + test("Pull out join condition contains other predicates") { + val udf = Substring("y.d".attr, 1, 5) + val joinType = Inner + val originalQuery = x.join(y, joinType, Option("x.a".attr === udf && "x.b".attr > "y.e".attr)) + .select("x.a".attr, "y.e".attr) + val correctAnswer = + x.select("x.a".attr, "x.b".attr, "x.c".attr).join( + y.select("y.d".attr, "y.e".attr, Alias(udf, udf.sql)()), + joinType, Option("x.a".attr === s"`${udf.sql}`".attr && "x.b".attr > "y.e".attr)) .select("x.a".attr, "y.e".attr) - val correctAnswer = - x.select("x.a".attr, "x.b".attr, "x.c".attr).join( - y.select("y.d".attr, "y.e".attr, Alias(udf, "_right_complex_expr0")()), - joinType, Option("x.a".attr === "_right_complex_expr0".attr && "x.b".attr > "y.e".attr)) - .select("x.a".attr, "y.e".attr) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) - } + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } - test("Push down EqualNullSafe join condition") { + test("Pull out EqualNullSafe join condition") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val joinType = Inner val udf = "x.b".attr + 1 + val coalesce1 = Coalesce(Seq(udf, Literal(0))) + val coalesce2 = Coalesce(Seq("y.e".attr, Literal(0))) + val isNull1 = IsNull(udf) + val isNull2 = IsNull("y.e".attr) + val originalQuery = x.join(y, joinType, Option(udf <=> "y.e".attr)) .select("x.a".attr, "y.e".attr) val correctAnswer = x.select("x.a".attr, "x.b".attr, "x.c".attr, - Alias(Coalesce(Seq(udf, Literal(0))), "_left_complex_expr0")(), - Alias(IsNull(udf), "_left_complex_expr1")()).join( + Alias(coalesce1, coalesce1.sql)(), + Alias(isNull1, isNull1.sql)()).join( y.select("y.d".attr, "y.e".attr, - Alias(Coalesce(Seq("y.e".attr, Literal(0))), "_right_complex_expr0")(), - Alias(IsNull("y.e".attr), "_right_complex_expr1")()), - joinType, Option("_left_complex_expr0".attr === "_right_complex_expr0".attr && - "_left_complex_expr1".attr === "_right_complex_expr1".attr)) + Alias(coalesce2, coalesce2.sql)(), + Alias(isNull2, isNull2.sql)()), + joinType, Option(s"`${coalesce1.sql}`".attr === s"`${coalesce2.sql}`".attr && + s"`${isNull1.sql}`".attr === s"`${isNull2.sql}`".attr)) .select("x.a".attr, "y.e".attr) comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) @@ -108,43 +107,27 @@ class PullOutJoinConditionSuite extends PlanTest { } test("Negative case: non-equality join keys") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val joinType = Inner - val udf = "x.b".attr + 1 - val originalQuery = x.join(y, joinType, Option(udf > "y.e".attr)) - .select("x.a".attr, "y.e".attr) + val joinType = Inner + val udf = "x.b".attr + 1 + val originalQuery = x.join(y, joinType, Option(udf > "y.e".attr)) + .select("x.a".attr, "y.e".attr) - comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) - } + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) } test("Negative case: all children are Attributes") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val condition = Option("x.a".attr === "y.d".attr) - val originalQuery = x.join(y, Inner, condition) - val correctAnswer = x.join(y, Inner, condition) + val condition = Option("x.a".attr === "y.d".attr) + val originalQuery = x.join(y, Inner, condition) + val correctAnswer = x.join(y, Inner, condition) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) - } + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } test("Negative case: contains Literal") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val condition = Option("x.a".attr === "string") - val originalQuery = x.join(y, Inner, condition) - val correctAnswer = x.join(y, Inner, condition) - - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) - } - } + val condition = Option("x.a".attr === "string") + val originalQuery = x.join(y, Inner, condition) + val correctAnswer = x.join(y, Inner, condition) - test("Negative case: BroadcastHashJoin") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10MB") { - Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => - val originalQuery = x.join(y, Inner, Option("x.a".attr === udf)) - .select("x.a".attr, "y.e".attr) - comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) - } - } + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 11b7ee65dad52..4f147dc716445 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -28,10 +28,10 @@ import org.apache.commons.io.FileUtils import org.apache.spark.{AccumulatorSuite, SparkException} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} -import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.sql.catalyst.expressions.{Cast, Coalesce, GenericRow, IsNotNull} import org.apache.spark.sql.catalyst.expressions.aggregate.{Complete, Partial} import org.apache.spark.sql.catalyst.optimizer.{ConvertToLocalRelation, NestedColumnAliasingSuite} -import org.apache.spark.sql.catalyst.plans.logical.{LocalLimit, Project, RepartitionByExpression, Sort} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalLimit, Project, RepartitionByExpression, Sort} import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.execution.{CommandResultExec, UnionExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -4211,6 +4211,31 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark sql("SELECT * FROM testData, LATERAL (SELECT * FROM testData)").collect() } } + + test("SPARK-36290: Pull out join condition can infer more filter conditions") { + import org.apache.spark.sql.catalyst.dsl.expressions.DslString + + withTable("t1", "t2") { + spark.sql("CREATE TABLE t1(a int, b int) using parquet") + spark.sql("CREATE TABLE t2(a string, b string, c string) using parquet") + + spark.sql("SELECT t1.* FROM t1 RIGHT JOIN t2 ON coalesce(t1.a, t1.b) = t2.a") + .queryExecution.optimizedPlan.find(_.isInstanceOf[Filter]) match { + case Some(Filter(condition, _)) => + condition === IsNotNull(Coalesce(Seq("a".attr, "b".attr))) + case _ => + fail("It should contains Filter") + } + + spark.sql("SELECT t1.* FROM t1 LEFT JOIN t2 ON t1.a = t2.a") + .queryExecution.optimizedPlan.find(_.isInstanceOf[Filter]) match { + case Some(Filter(condition, _)) => + condition === IsNotNull(Cast("a".attr, IntegerType)) + case _ => + fail("It should contains Filter") + } + } + } } case class Foo(bar: Option[String]) From 4aebe0a19fdf84516e30ab917eb149d39c4d4ac6 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 5 Nov 2021 12:36:46 +0800 Subject: [PATCH 16/17] Fix test error --- .../org/apache/spark/sql/DataFrameJoinSuite.scala | 6 +++--- .../test/scala/org/apache/spark/sql/JoinSuite.scala | 2 +- .../spark/sql/execution/WholeStageCodegenSuite.scala | 11 ++++++----- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index a803fa88ed313..2144976452fb6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -280,9 +280,9 @@ class DataFrameJoinSuite extends QueryTest plan match { // SPARK-34178: we can't match the plan before the fix due to // the right side plan doesn't contains dataset id. - case Join( - LogicalPlanWithDatasetId(_, leftId), - LogicalPlanWithDatasetId(_, rightId), _, _, _) => + case Project(_, Join( + Project(_, LogicalPlanWithDatasetId(_, leftId)), + Project(_, LogicalPlanWithDatasetId(_, rightId)), _, _, _)) => assert(leftId === rightId) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 77493afe43145..10b49a6efc76d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -1057,7 +1057,7 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val pythonEvals = collect(joinNode.get) { case p: BatchEvalPythonExec => p } - assert(pythonEvals.size == 2) + assert(pythonEvals.size == 4) checkAnswer(df, Row(1, 2, 1, 2) :: Nil) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 7da813cfdab6f..31cf8e23934ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -143,9 +143,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val schema = new StructType().add("k", IntegerType).add("v", StringType) val smallDF = spark.createDataFrame(rdd, schema) val df = spark.range(10).join(broadcast(smallDF), col("k") === col("id")) - assert(df.queryExecution.executedPlan.find(p => - p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[BroadcastHashJoinExec]).isDefined) + val broadcastHashJoin = df.queryExecution.executedPlan.find { + case WholeStageCodegenExec(ProjectExec(_, _: BroadcastHashJoinExec)) => true + } + assert(broadcastHashJoin.isDefined) assert(df.collect() === Array(Row(1, 1, "1"), Row(1, 1, "1"), Row(2, 2, "2"))) } @@ -187,7 +188,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession // test one join with non-unique key from build side val joinNonUniqueDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2" % 3, "full_outer") assert(joinNonUniqueDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true + case WholeStageCodegenExec(ProjectExec(_, _: ShuffledHashJoinExec)) => true }.size === 1) checkAnswer(joinNonUniqueDF, Seq(Row(0, 0), Row(0, 3), Row(0, 6), Row(0, 9), Row(1, 1), Row(1, 4), Row(1, 7), Row(2, 2), Row(2, 5), Row(2, 8), Row(3, null), Row(4, null))) @@ -196,7 +197,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val joinWithNonEquiDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2" % 3 && $"k1" + 3 =!= $"k2", "full_outer") assert(joinWithNonEquiDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true + case WholeStageCodegenExec(ProjectExec(_, _: ShuffledHashJoinExec)) => true }.size === 1) checkAnswer(joinWithNonEquiDF, Seq(Row(0, 0), Row(0, 6), Row(0, 9), Row(1, 1), Row(1, 7), Row(2, 2), Row(2, 8), Row(3, null), Row(4, null), Row(null, 3), Row(null, 4), From c6fca9aad5b3e1d819d35441dd30c6e12de04da2 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 5 Nov 2021 23:19:01 +0800 Subject: [PATCH 17/17] Address comments --- .../optimizer/PullOutJoinConditionSuite.scala | 90 ++++++------------- 1 file changed, 27 insertions(+), 63 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala index eeb85b905d313..7f120362779c7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutJoinConditionSuite.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Coalesce, IsNull, Liter import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.internal.SQLConf class PullOutJoinConditionSuite extends PlanTest { @@ -40,94 +39,59 @@ class PullOutJoinConditionSuite extends PlanTest { private val y = testRelation1.subquery('y) test("Pull out join keys evaluation(String expressions)") { - val joinType = Inner Seq(Upper("y.d".attr), Substring("y.d".attr, 1, 5)).foreach { udf => - val originalQuery = x.join(y, joinType, Option("x.a".attr === udf)) - .select("x.a".attr, "y.e".attr) - val correctAnswer = x.select("x.a".attr, "x.b".attr, "x.c".attr) - .join(y.select("y.d".attr, "y.e".attr, Alias(udf, udf.sql)()), - joinType, Option("x.a".attr === s"`${udf.sql}`".attr)).select("x.a".attr, "y.e".attr) + val originalQuery = x.join(y, condition = Option('a === udf)).select('a, 'e) + val correctAnswer = x.select('a, 'b, 'c) + .join(y.select('d, 'e, Alias(udf, udf.sql)()), + condition = Option('a === s"`${udf.sql}`".attr)).select('a, 'e) comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } } - test("Pull out join keys evaluation(null expressions)") { - val joinType = Inner - val udf = Coalesce(Seq("x.b".attr, "x.c".attr)) - val originalQuery = x.join(y, joinType, Option(udf === "y.e".attr)) - .select("x.a".attr, "y.e".attr) - val correctAnswer = - x.select("x.a".attr, "x.b".attr, "x.c".attr, Alias(udf, udf.sql)()).join( - y.select("y.d".attr, "y.e".attr), - joinType, Option(s"`${udf.sql}`".attr === "y.e".attr)) - .select("x.a".attr, "y.e".attr) - - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) - } - test("Pull out join condition contains other predicates") { val udf = Substring("y.d".attr, 1, 5) - val joinType = Inner - val originalQuery = x.join(y, joinType, Option("x.a".attr === udf && "x.b".attr > "y.e".attr)) - .select("x.a".attr, "y.e".attr) - val correctAnswer = - x.select("x.a".attr, "x.b".attr, "x.c".attr).join( - y.select("y.d".attr, "y.e".attr, Alias(udf, udf.sql)()), - joinType, Option("x.a".attr === s"`${udf.sql}`".attr && "x.b".attr > "y.e".attr)) - .select("x.a".attr, "y.e".attr) + val originalQuery = x.join(y, condition = Option('a === udf && 'b > 'e)).select('a, 'e) + val correctAnswer = x.select('a, 'b, 'c) + .join(y.select('d, 'e, Alias(udf, udf.sql)()), + condition = Option('a === s"`${udf.sql}`".attr && 'b > 'e)).select('a, 'e) comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } test("Pull out EqualNullSafe join condition") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val joinType = Inner - val udf = "x.b".attr + 1 - val coalesce1 = Coalesce(Seq(udf, Literal(0))) - val coalesce2 = Coalesce(Seq("y.e".attr, Literal(0))) - val isNull1 = IsNull(udf) - val isNull2 = IsNull("y.e".attr) - - val originalQuery = x.join(y, joinType, Option(udf <=> "y.e".attr)) - .select("x.a".attr, "y.e".attr) - val correctAnswer = - x.select("x.a".attr, "x.b".attr, "x.c".attr, - Alias(coalesce1, coalesce1.sql)(), - Alias(isNull1, isNull1.sql)()).join( - y.select("y.d".attr, "y.e".attr, - Alias(coalesce2, coalesce2.sql)(), - Alias(isNull2, isNull2.sql)()), - joinType, Option(s"`${coalesce1.sql}`".attr === s"`${coalesce2.sql}`".attr && - s"`${isNull1.sql}`".attr === s"`${isNull2.sql}`".attr)) - .select("x.a".attr, "y.e".attr) + val joinType = Inner + val udf = "x.b".attr + 1 + val coalesce1 = Coalesce(Seq(udf, Literal(0))) + val coalesce2 = Coalesce(Seq("y.e".attr, Literal(0))) + val isNull1 = IsNull(udf) + val isNull2 = IsNull("y.e".attr) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) - } + val originalQuery = x.join(y, joinType, Option(udf <=> 'e)).select('a, 'e) + val correctAnswer = + x.select('a, 'b, 'c, Alias(coalesce1, coalesce1.sql)(), Alias(isNull1, isNull1.sql)()) + .join(y.select('d, 'e, Alias(coalesce2, coalesce2.sql)(), Alias(isNull2, isNull2.sql)()), + condition = Option(s"`${coalesce1.sql}`".attr === s"`${coalesce2.sql}`".attr && + s"`${isNull1.sql}`".attr === s"`${isNull2.sql}`".attr)).select('a, 'e) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } test("Negative case: non-equality join keys") { - val joinType = Inner - val udf = "x.b".attr + 1 - val originalQuery = x.join(y, joinType, Option(udf > "y.e".attr)) - .select("x.a".attr, "y.e".attr) + val originalQuery = x.join(y, condition = Option("x.b".attr + 1 > 'e)).select('a, 'e) comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) } test("Negative case: all children are Attributes") { - val condition = Option("x.a".attr === "y.d".attr) - val originalQuery = x.join(y, Inner, condition) - val correctAnswer = x.join(y, Inner, condition) + val originalQuery = x.join(y, condition = Option('a === 'd)) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) } test("Negative case: contains Literal") { - val condition = Option("x.a".attr === "string") - val originalQuery = x.join(y, Inner, condition) - val correctAnswer = x.join(y, Inner, condition) + val originalQuery = x.join(y, condition = Option('a === "string")) - comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + comparePlans(Optimize.execute(originalQuery.analyze), originalQuery.analyze) } }