diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 852055c4df16..57b28382df09 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -978,10 +978,8 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB case ScalarSubquery(query, outerAttrs, _, _, _, _) => // Scalar subquery must return one column as output. if (query.output.size != 1) { - expr.failAnalysis( - errorClass = "INVALID_SUBQUERY_EXPRESSION." + - "SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", - messageParameters = Map("number" -> query.output.size.toString)) + throw QueryCompilationErrors.subqueryReturnMoreThanOneColumn(query.output.size, + expr.origin) } if (outerAttrs.nonEmpty) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index 1b43922de278..39dda9a13dad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -82,6 +82,13 @@ object DeduplicateRelations extends Rule[LogicalPlan] { } } + private def existDuplicatedExprId( + existingRelations: mutable.HashSet[RelationWrapper], + plan: RelationWrapper): Boolean = { + existingRelations.filter(_.cls == plan.cls) + .exists(_.outputAttrIds.intersect(plan.outputAttrIds).nonEmpty) + } + /** * Deduplicate any duplicated relations of a LogicalPlan * @param existingRelations the known unique relations for a LogicalPlan @@ -95,59 +102,161 @@ object DeduplicateRelations extends Rule[LogicalPlan] { case p: LogicalPlan if p.isStreaming => (plan, false) case m: MultiInstanceRelation => - val planWrapper = RelationWrapper(m.getClass, m.output.map(_.exprId.id)) - if (existingRelations.contains(planWrapper)) { - val newNode = m.newInstance() - newNode.copyTagsFrom(m) - (newNode, true) - } else { - existingRelations.add(planWrapper) - (m, false) - } + deduplicateAndRenew[LogicalPlan with MultiInstanceRelation]( + existingRelations, + m, + _.output.map(_.exprId.id), + node => node.newInstance().asInstanceOf[LogicalPlan with MultiInstanceRelation]) + + case p: Project => + deduplicateAndRenew[Project]( + existingRelations, + p, + newProject => findAliases(newProject.projectList).map(_.exprId.id).toSeq, + newProject => newProject.copy(newAliases(newProject.projectList))) + + case s: SerializeFromObject => + deduplicateAndRenew[SerializeFromObject]( + existingRelations, + s, + _.serializer.map(_.exprId.id), + newSer => newSer.copy(newSer.serializer.map(_.newInstance()))) + + case f: FlatMapGroupsInPandas => + deduplicateAndRenew[FlatMapGroupsInPandas]( + existingRelations, + f, + _.output.map(_.exprId.id), + newFlatMap => newFlatMap.copy(output = newFlatMap.output.map(_.newInstance()))) + + case f: FlatMapCoGroupsInPandas => + deduplicateAndRenew[FlatMapCoGroupsInPandas]( + existingRelations, + f, + _.output.map(_.exprId.id), + newFlatMap => newFlatMap.copy(output = newFlatMap.output.map(_.newInstance()))) + + case m: MapInPandas => + deduplicateAndRenew[MapInPandas]( + existingRelations, + m, + _.output.map(_.exprId.id), + newMap => newMap.copy(output = newMap.output.map(_.newInstance()))) + + case p: PythonMapInArrow => + deduplicateAndRenew[PythonMapInArrow]( + existingRelations, + p, + _.output.map(_.exprId.id), + newMap => newMap.copy(output = newMap.output.map(_.newInstance()))) + + case a: AttachDistributedSequence => + deduplicateAndRenew[AttachDistributedSequence]( + existingRelations, + a, + _.producedAttributes.map(_.exprId.id).toSeq, + newAttach => newAttach.copy(sequenceAttr = newAttach.producedAttributes + .map(_.newInstance()).head)) + + case g: Generate => + deduplicateAndRenew[Generate]( + existingRelations, + g, + _.generatorOutput.map(_.exprId.id), newGenerate => + newGenerate.copy(generatorOutput = newGenerate.generatorOutput.map(_.newInstance()))) + + case e: Expand => + deduplicateAndRenew[Expand]( + existingRelations, + e, + _.producedAttributes.map(_.exprId.id).toSeq, + newExpand => newExpand.copy(output = newExpand.output.map(_.newInstance()))) + + case w: Window => + deduplicateAndRenew[Window]( + existingRelations, + w, + _.windowExpressions.map(_.exprId.id), + newWindow => newWindow.copy(windowExpressions = + newWindow.windowExpressions.map(_.newInstance()))) + + case s: ScriptTransformation => + deduplicateAndRenew[ScriptTransformation]( + existingRelations, + s, + _.output.map(_.exprId.id), + newScript => newScript.copy(output = newScript.output.map(_.newInstance()))) case plan: LogicalPlan => - var planChanged = false - val newPlan = if (plan.children.nonEmpty) { - val newChildren = mutable.ArrayBuffer.empty[LogicalPlan] - for (c <- plan.children) { - val (renewed, changed) = renewDuplicatedRelations(existingRelations, c) - newChildren += renewed - if (changed) { - planChanged = true - } - } + deduplicate(existingRelations, plan) + } - val planWithNewSubquery = plan.transformExpressions { - case subquery: SubqueryExpression => - val (renewed, changed) = renewDuplicatedRelations(existingRelations, subquery.plan) - if (changed) planChanged = true - subquery.withNewPlan(renewed) + private def deduplicate( + existingRelations: mutable.HashSet[RelationWrapper], + plan: LogicalPlan): (LogicalPlan, Boolean) = { + var planChanged = false + val newPlan = if (plan.children.nonEmpty) { + val newChildren = mutable.ArrayBuffer.empty[LogicalPlan] + for (c <- plan.children) { + val (renewed, changed) = renewDuplicatedRelations(existingRelations, c) + newChildren += renewed + if (changed) { + planChanged = true } + } + + val planWithNewSubquery = plan.transformExpressions { + case subquery: SubqueryExpression => + val (renewed, changed) = renewDuplicatedRelations(existingRelations, subquery.plan) + if (changed) planChanged = true + subquery.withNewPlan(renewed) + } - if (planChanged) { - if (planWithNewSubquery.childrenResolved) { - val planWithNewChildren = planWithNewSubquery.withNewChildren(newChildren.toSeq) - val attrMap = AttributeMap( - plan - .children - .flatMap(_.output).zip(newChildren.flatMap(_.output)) - .filter { case (a1, a2) => a1.exprId != a2.exprId } - ) - if (attrMap.isEmpty) { - planWithNewChildren - } else { - planWithNewChildren.rewriteAttrs(attrMap) - } + if (planChanged) { + if (planWithNewSubquery.childrenResolved) { + val planWithNewChildren = planWithNewSubquery.withNewChildren(newChildren.toSeq) + val attrMap = AttributeMap(plan.children.flatMap(_.output) + .zip(newChildren.flatMap(_.output)).filter { case (a1, a2) => a1.exprId != a2.exprId }) + if (attrMap.isEmpty) { + planWithNewChildren } else { - planWithNewSubquery.withNewChildren(newChildren.toSeq) + planWithNewChildren.rewriteAttrs(attrMap) } } else { - plan + planWithNewSubquery.withNewChildren(newChildren.toSeq) } } else { plan } + } else { + plan + } + (newPlan, planChanged) + } + + private def deduplicateAndRenew[T <: LogicalPlan]( + existingRelations: mutable.HashSet[RelationWrapper], plan: T, + getExprIds: T => Seq[Long], + copyNewPlan: T => T): (LogicalPlan, Boolean) = { + var (newPlan, planChanged) = deduplicate(existingRelations, plan) + if (newPlan.resolved) { + val exprIds = getExprIds(newPlan.asInstanceOf[T]) + if (exprIds.nonEmpty) { + val planWrapper = RelationWrapper(newPlan.getClass, exprIds) + if (existDuplicatedExprId(existingRelations, planWrapper)) { + newPlan = copyNewPlan(newPlan.asInstanceOf[T]) + newPlan.copyTagsFrom(plan) + (newPlan, true) + } else { + existingRelations.add(planWrapper) + (newPlan, planChanged) + } + } else { + (newPlan, planChanged) + } + } else { (newPlan, planChanged) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index cd31b94905a2..bbcd9e336b45 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -271,7 +271,10 @@ case class ScalarSubquery( mayHaveCountBug: Option[Boolean] = None) extends SubqueryExpression(plan, outerAttrs, exprId, joinCond, hint) with Unevaluable { override def dataType: DataType = { - assert(plan.schema.fields.nonEmpty, "Scalar subquery should have only one column") + if (!plan.schema.fields.nonEmpty) { + throw QueryCompilationErrors.subqueryReturnMoreThanOneColumn(plan.schema.fields.length, + origin) + } plan.schema.fields.head.dataType } override def nullable: Boolean = true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index aab5ada6e4dd..1fccd15bdc4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2000,6 +2000,14 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { messageParameters = Map("function" -> funcStr)) } + def subqueryReturnMoreThanOneColumn(number: Int, origin: Origin): Throwable = { + new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION." + + "SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", + origin = origin, + messageParameters = Map("number" -> number.toString)) + } + def unsupportedCorrelatedReferenceDataTypeError( expr: Expression, dataType: DataType, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala index 04171a85eec6..ba7386f8c9b5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala @@ -437,25 +437,6 @@ class LeftSemiAntiJoinPushDownSuite extends PlanTest { } } - Seq(LeftSemi, LeftAnti).foreach { case jt => - test(s"Aggregate: $jt join no pushdown - join condition refers left leg and right leg child") { - val aggregation = testRelation - .select($"b".as("id"), $"c") - .groupBy($"id")($"id", sum($"c").as("sum")) - - // reference "b" exists in left leg, and the children of the right leg of the join - val originalQuery = aggregation.select(($"id" + 1).as("id_plus_1"), $"sum") - .join(aggregation, joinType = jt, condition = Some($"id" === $"id_plus_1")) - val optimized = Optimize.execute(originalQuery.analyze) - val correctAnswer = testRelation - .select($"b".as("id"), $"c") - .groupBy($"id")(($"id" + 1).as("id_plus_1"), sum($"c").as("sum")) - .join(aggregation, joinType = jt, condition = Some($"id" === $"id_plus_1")) - .analyze - comparePlans(optimized, correctAnswer) - } - } - Seq(LeftSemi, LeftAnti).foreach { case outerJT => Seq(Inner, LeftOuter, RightOuter, Cross).foreach { case innerJT => test(s"$outerJT no pushdown - join condition refers none of the leg - join type $innerJT") { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out index a9bfbc69cf24..96b6749b8670 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out @@ -1977,17 +1977,16 @@ Union false, false : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] : +- LocalRelation [col1#x, col2#x] -+- Project [c1#x AS c1#x, c2#x AS c2#x, c1#x AS c1#x, c2#x AS c2#x] - +- Project [c1#x, c2#x, c1#x, c2#x] - +- Join Inner - :- SubqueryAlias spark_catalog.default.t1 - : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) - : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] - : +- LocalRelation [col1#x, col2#x] - +- SubqueryAlias spark_catalog.default.t4 - +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) - +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] - +- LocalRelation [col1#x, col2#x] ++- Project [c1#x, c2#x, c1#x, c2#x] + +- Join Inner + :- SubqueryAlias spark_catalog.default.t1 + : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t4 + +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] -- !query @@ -2030,27 +2029,26 @@ Union false, false : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] : +- LocalRelation [col1#x, col2#x] -+- Project [c1#x AS c1#x, c2#x AS c2#x, c2#x AS c2#x] - +- Project [c1#x, c2#x, c2#x] - +- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner - : +- SubqueryAlias __auto_generated_subquery_name - : +- Union false, false - : :- Project [c2#x] - : : +- Filter (outer(c1#x) <= c1#x) - : : +- SubqueryAlias spark_catalog.default.t1 - : : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) - : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] - : : +- LocalRelation [col1#x, col2#x] - : +- Project [c2#x] - : +- Filter (c1#x < outer(c1#x)) - : +- SubqueryAlias spark_catalog.default.t4 - : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) - : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] - : +- LocalRelation [col1#x, col2#x] - +- SubqueryAlias spark_catalog.default.t2 - +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) - +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] - +- LocalRelation [col1#x, col2#x] ++- Project [c1#x, c2#x, c2#x] + +- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Project [c2#x] + : : +- Filter (outer(c1#x) <= c1#x) + : : +- SubqueryAlias spark_catalog.default.t1 + : : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- Filter (c1#x < outer(c1#x)) + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t2 + +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out index cb41f7cdc455..18c879466e37 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -1057,3 +1057,21 @@ Project [c1#xL, c2#xL] : +- Range (1, 2, step=1, splits=None) +- SubqueryAlias t1 +- Range (1, 3, step=1, splits=None) + + +-- !query +SELECT c1, c2, (SELECT count(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1 HAVING cnt = 0) FROM t1 +-- !query analysis +Project [c1#x, c2#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] +: +- Filter (cnt#xL = cast(0 as bigint)) +: +- Aggregate [count(1) AS cnt#xL] +: +- Filter (outer(c1#x) = c1#x) +: +- SubqueryAlias t2 +: +- SubqueryAlias t1 +: +- View (`t1`, [c1#x,c2#x]) +: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- SubqueryAlias t1 + +- View (`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql index 741292d2c0fa..e4f7b25a1684 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql @@ -255,3 +255,6 @@ select * from ( where t1.id = t2.id ) c2 from range (1, 3) t1 ) t where t.c2 is not null; + +-- SPARK-43838: Subquery on single table with having clause +SELECT c1, c2, (SELECT count(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1 HAVING cnt = 0) FROM t1 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out index de7fa9747621..85bd9137602a 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -598,3 +598,12 @@ where t.c2 is not null struct -- !query output 1 1 + + +-- !query +SELECT c1, c2, (SELECT count(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1 HAVING cnt = 0) FROM t1 +-- !query schema +struct +-- !query output +0 1 NULL +1 2 NULL 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 c638699b2756..91dbbbc0c8ee 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 @@ -171,123 +171,123 @@ ReadSchema: struct Input [2]: [ws_ext_sales_price#52, ws_sold_date_sk#53] (26) Project [codegen id : 6] -Output [2]: [ws_sold_date_sk#53 AS sold_date_sk#3, ws_ext_sales_price#52 AS sales_price#4] +Output [2]: [ws_sold_date_sk#53 AS sold_date_sk#54, ws_ext_sales_price#52 AS sales_price#55] Input [2]: [ws_ext_sales_price#52, ws_sold_date_sk#53] (27) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#54, cs_sold_date_sk#55] +Output [2]: [cs_ext_sales_price#56, cs_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#55)] +PartitionFilters: [isnotnull(cs_sold_date_sk#57)] ReadSchema: struct (28) ColumnarToRow [codegen id : 7] -Input [2]: [cs_ext_sales_price#54, cs_sold_date_sk#55] +Input [2]: [cs_ext_sales_price#56, cs_sold_date_sk#57] (29) Project [codegen id : 7] -Output [2]: [cs_sold_date_sk#55 AS sold_date_sk#7, cs_ext_sales_price#54 AS sales_price#8] -Input [2]: [cs_ext_sales_price#54, cs_sold_date_sk#55] +Output [2]: [cs_sold_date_sk#57 AS sold_date_sk#58, cs_ext_sales_price#56 AS sales_price#59] +Input [2]: [cs_ext_sales_price#56, cs_sold_date_sk#57] (30) Union (31) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#56, d_week_seq#57, d_day_name#58] +Output [3]: [d_date_sk#60, d_week_seq#61, d_day_name#62] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] -Input [3]: [d_date_sk#56, d_week_seq#57, d_day_name#58] +Input [3]: [d_date_sk#60, d_week_seq#61, d_day_name#62] (33) Filter [codegen id : 8] -Input [3]: [d_date_sk#56, d_week_seq#57, d_day_name#58] -Condition : ((isnotnull(d_date_sk#56) AND isnotnull(d_week_seq#57)) AND might_contain(Subquery scalar-subquery#59, [id=#60], xxhash64(d_week_seq#57, 42))) +Input [3]: [d_date_sk#60, d_week_seq#61, d_day_name#62] +Condition : ((isnotnull(d_date_sk#60) AND isnotnull(d_week_seq#61)) AND might_contain(Subquery scalar-subquery#63, [id=#64], xxhash64(d_week_seq#61, 42))) (34) BroadcastExchange -Input [3]: [d_date_sk#56, d_week_seq#57, d_day_name#58] +Input [3]: [d_date_sk#60, d_week_seq#61, d_day_name#62] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] (35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [sold_date_sk#3] -Right keys [1]: [d_date_sk#56] +Left keys [1]: [sold_date_sk#54] +Right keys [1]: [d_date_sk#60] Join type: Inner Join condition: None (36) Project [codegen id : 9] -Output [3]: [sales_price#4, d_week_seq#57, d_day_name#58] -Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#56, d_week_seq#57, d_day_name#58] +Output [3]: [sales_price#55, d_week_seq#61, d_day_name#62] +Input [5]: [sold_date_sk#54, sales_price#55, d_date_sk#60, d_week_seq#61, d_day_name#62] (37) HashAggregate [codegen id : 9] -Input [3]: [sales_price#4, d_week_seq#57, d_day_name#58] -Keys [1]: [d_week_seq#57] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum#61, sum#62, sum#63, sum#64, sum#65, sum#66, sum#67] -Results [8]: [d_week_seq#57, sum#68, sum#69, sum#70, sum#71, sum#72, sum#73, sum#74] +Input [3]: [sales_price#55, d_week_seq#61, d_day_name#62] +Keys [1]: [d_week_seq#61] +Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Sunday ) THEN sales_price#55 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Monday ) THEN sales_price#55 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Tuesday ) THEN sales_price#55 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Wednesday) THEN sales_price#55 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Thursday ) THEN sales_price#55 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Friday ) THEN sales_price#55 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#62 = Saturday ) THEN sales_price#55 END))] +Aggregate Attributes [7]: [sum#65, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] +Results [8]: [d_week_seq#61, sum#72, sum#73, sum#74, sum#75, sum#76, sum#77, sum#78] (38) Exchange -Input [8]: [d_week_seq#57, sum#68, sum#69, sum#70, sum#71, sum#72, sum#73, sum#74] -Arguments: hashpartitioning(d_week_seq#57, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [8]: [d_week_seq#61, sum#72, sum#73, sum#74, sum#75, sum#76, sum#77, sum#78] +Arguments: hashpartitioning(d_week_seq#61, 5), ENSURE_REQUIREMENTS, [plan_id=5] (39) HashAggregate [codegen id : 11] -Input [8]: [d_week_seq#57, sum#68, sum#69, sum#70, sum#71, sum#72, sum#73, sum#74] -Keys [1]: [d_week_seq#57] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#4 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#4 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#4 END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#4 END))#33, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#4 END))#34] -Results [8]: [d_week_seq#57, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#4 END))#28,17,2) AS sun_sales#75, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#4 END))#29,17,2) AS mon_sales#76, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#4 END))#30,17,2) AS tue_sales#77, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#4 END))#31,17,2) AS wed_sales#78, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#4 END))#32,17,2) AS thu_sales#79, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#4 END))#33,17,2) AS fri_sales#80, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#4 END))#34,17,2) AS sat_sales#81] +Input [8]: [d_week_seq#61, sum#72, sum#73, sum#74, sum#75, sum#76, sum#77, sum#78] +Keys [1]: [d_week_seq#61] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#62 = Sunday ) THEN sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#62 = Monday ) THEN sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#62 = Tuesday ) THEN sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#62 = Wednesday) THEN sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#62 = Thursday ) THEN sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#62 = Friday ) THEN sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#62 = Saturday ) THEN sales_price#55 END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#62 = Sunday ) THEN sales_price#55 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#62 = Monday ) THEN sales_price#55 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#62 = Tuesday ) THEN sales_price#55 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#62 = Wednesday) THEN sales_price#55 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#62 = Thursday ) THEN sales_price#55 END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#62 = Friday ) THEN sales_price#55 END))#33, sum(UnscaledValue(CASE WHEN (d_day_name#62 = Saturday ) THEN sales_price#55 END))#34] +Results [8]: [d_week_seq#61, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Sunday ) THEN sales_price#55 END))#28,17,2) AS sun_sales#79, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Monday ) THEN sales_price#55 END))#29,17,2) AS mon_sales#80, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Tuesday ) THEN sales_price#55 END))#30,17,2) AS tue_sales#81, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Wednesday) THEN sales_price#55 END))#31,17,2) AS wed_sales#82, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Thursday ) THEN sales_price#55 END))#32,17,2) AS thu_sales#83, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Friday ) THEN sales_price#55 END))#33,17,2) AS fri_sales#84, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#62 = Saturday ) THEN sales_price#55 END))#34,17,2) AS sat_sales#85] (40) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#82, d_year#83] +Output [2]: [d_week_seq#86, d_year#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [2]: [d_week_seq#82, d_year#83] +Input [2]: [d_week_seq#86, d_year#87] (42) Filter [codegen id : 10] -Input [2]: [d_week_seq#82, d_year#83] -Condition : ((isnotnull(d_year#83) AND (d_year#83 = 2002)) AND isnotnull(d_week_seq#82)) +Input [2]: [d_week_seq#86, d_year#87] +Condition : ((isnotnull(d_year#87) AND (d_year#87 = 2002)) AND isnotnull(d_week_seq#86)) (43) Project [codegen id : 10] -Output [1]: [d_week_seq#82] -Input [2]: [d_week_seq#82, d_year#83] +Output [1]: [d_week_seq#86] +Input [2]: [d_week_seq#86, d_year#87] (44) BroadcastExchange -Input [1]: [d_week_seq#82] +Input [1]: [d_week_seq#86] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] (45) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#57] -Right keys [1]: [d_week_seq#82] +Left keys [1]: [d_week_seq#61] +Right keys [1]: [d_week_seq#86] Join type: Inner Join condition: None (46) Project [codegen id : 11] -Output [8]: [d_week_seq#57 AS d_week_seq2#84, sun_sales#75 AS sun_sales2#85, mon_sales#76 AS mon_sales2#86, tue_sales#77 AS tue_sales2#87, wed_sales#78 AS wed_sales2#88, thu_sales#79 AS thu_sales2#89, fri_sales#80 AS fri_sales2#90, sat_sales#81 AS sat_sales2#91] -Input [9]: [d_week_seq#57, sun_sales#75, mon_sales#76, tue_sales#77, wed_sales#78, thu_sales#79, fri_sales#80, sat_sales#81, d_week_seq#82] +Output [8]: [d_week_seq#61 AS d_week_seq2#88, sun_sales#79 AS sun_sales2#89, mon_sales#80 AS mon_sales2#90, tue_sales#81 AS tue_sales2#91, wed_sales#82 AS wed_sales2#92, thu_sales#83 AS thu_sales2#93, fri_sales#84 AS fri_sales2#94, sat_sales#85 AS sat_sales2#95] +Input [9]: [d_week_seq#61, sun_sales#79, mon_sales#80, tue_sales#81, wed_sales#82, thu_sales#83, fri_sales#84, sat_sales#85, d_week_seq#86] (47) BroadcastExchange -Input [8]: [d_week_seq2#84, sun_sales2#85, mon_sales2#86, tue_sales2#87, wed_sales2#88, thu_sales2#89, fri_sales2#90, sat_sales2#91] +Input [8]: [d_week_seq2#88, sun_sales2#89, mon_sales2#90, tue_sales2#91, wed_sales2#92, thu_sales2#93, fri_sales2#94, sat_sales2#95] Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=7] (48) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#44] -Right keys [1]: [(d_week_seq2#84 - 53)] +Right keys [1]: [(d_week_seq2#88 - 53)] Join type: Inner Join condition: None (49) Project [codegen id : 12] -Output [8]: [d_week_seq1#44, round((sun_sales1#45 / sun_sales2#85), 2) AS round((sun_sales1 / sun_sales2), 2)#92, round((mon_sales1#46 / mon_sales2#86), 2) AS round((mon_sales1 / mon_sales2), 2)#93, round((tue_sales1#47 / tue_sales2#87), 2) AS round((tue_sales1 / tue_sales2), 2)#94, round((wed_sales1#48 / wed_sales2#88), 2) AS round((wed_sales1 / wed_sales2), 2)#95, round((thu_sales1#49 / thu_sales2#89), 2) AS round((thu_sales1 / thu_sales2), 2)#96, round((fri_sales1#50 / fri_sales2#90), 2) AS round((fri_sales1 / fri_sales2), 2)#97, round((sat_sales1#51 / sat_sales2#91), 2) AS round((sat_sales1 / sat_sales2), 2)#98] -Input [16]: [d_week_seq1#44, sun_sales1#45, mon_sales1#46, tue_sales1#47, wed_sales1#48, thu_sales1#49, fri_sales1#50, sat_sales1#51, d_week_seq2#84, sun_sales2#85, mon_sales2#86, tue_sales2#87, wed_sales2#88, thu_sales2#89, fri_sales2#90, sat_sales2#91] +Output [8]: [d_week_seq1#44, round((sun_sales1#45 / sun_sales2#89), 2) AS round((sun_sales1 / sun_sales2), 2)#96, round((mon_sales1#46 / mon_sales2#90), 2) AS round((mon_sales1 / mon_sales2), 2)#97, round((tue_sales1#47 / tue_sales2#91), 2) AS round((tue_sales1 / tue_sales2), 2)#98, round((wed_sales1#48 / wed_sales2#92), 2) AS round((wed_sales1 / wed_sales2), 2)#99, round((thu_sales1#49 / thu_sales2#93), 2) AS round((thu_sales1 / thu_sales2), 2)#100, round((fri_sales1#50 / fri_sales2#94), 2) AS round((fri_sales1 / fri_sales2), 2)#101, round((sat_sales1#51 / sat_sales2#95), 2) AS round((sat_sales1 / sat_sales2), 2)#102] +Input [16]: [d_week_seq1#44, sun_sales1#45, mon_sales1#46, tue_sales1#47, wed_sales1#48, thu_sales1#49, fri_sales1#50, sat_sales1#51, d_week_seq2#88, sun_sales2#89, mon_sales2#90, tue_sales2#91, wed_sales2#92, thu_sales2#93, fri_sales2#94, sat_sales2#95] (50) Exchange -Input [8]: [d_week_seq1#44, round((sun_sales1 / sun_sales2), 2)#92, round((mon_sales1 / mon_sales2), 2)#93, round((tue_sales1 / tue_sales2), 2)#94, round((wed_sales1 / wed_sales2), 2)#95, round((thu_sales1 / thu_sales2), 2)#96, round((fri_sales1 / fri_sales2), 2)#97, round((sat_sales1 / sat_sales2), 2)#98] +Input [8]: [d_week_seq1#44, round((sun_sales1 / sun_sales2), 2)#96, round((mon_sales1 / mon_sales2), 2)#97, round((tue_sales1 / tue_sales2), 2)#98, round((wed_sales1 / wed_sales2), 2)#99, round((thu_sales1 / thu_sales2), 2)#100, round((fri_sales1 / fri_sales2), 2)#101, round((sat_sales1 / sat_sales2), 2)#102] Arguments: rangepartitioning(d_week_seq1#44 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=8] (51) Sort [codegen id : 13] -Input [8]: [d_week_seq1#44, round((sun_sales1 / sun_sales2), 2)#92, round((mon_sales1 / mon_sales2), 2)#93, round((tue_sales1 / tue_sales2), 2)#94, round((wed_sales1 / wed_sales2), 2)#95, round((thu_sales1 / thu_sales2), 2)#96, round((fri_sales1 / fri_sales2), 2)#97, round((sat_sales1 / sat_sales2), 2)#98] +Input [8]: [d_week_seq1#44, round((sun_sales1 / sun_sales2), 2)#96, round((mon_sales1 / mon_sales2), 2)#97, round((tue_sales1 / tue_sales2), 2)#98, round((wed_sales1 / wed_sales2), 2)#99, round((thu_sales1 / thu_sales2), 2)#100, round((fri_sales1 / fri_sales2), 2)#101, round((sat_sales1 / sat_sales2), 2)#102] Arguments: [d_week_seq1#44 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -324,21 +324,21 @@ Input [2]: [d_week_seq#42, d_year#43] Input [1]: [d_week_seq#42] Keys: [] Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [buf#99] -Results [1]: [buf#100] +Aggregate Attributes [1]: [buf#103] +Results [1]: [buf#104] (57) Exchange -Input [1]: [buf#100] +Input [1]: [buf#104] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] (58) ObjectHashAggregate -Input [1]: [buf#100] +Input [1]: [buf#104] Keys: [] Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)#101] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)#101 AS bloomFilter#102] +Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)#105] +Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#42, 42), 362, 9656, 0, 0)#105 AS bloomFilter#106] -Subquery:2 Hosting operator id = 33 Hosting Expression = Subquery scalar-subquery#59, [id=#60] +Subquery:2 Hosting operator id = 33 Hosting Expression = Subquery scalar-subquery#63, [id=#64] ObjectHashAggregate (65) +- Exchange (64) +- ObjectHashAggregate (63) @@ -349,39 +349,39 @@ ObjectHashAggregate (65) (59) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#82, d_year#83] +Output [2]: [d_week_seq#86, d_year#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct (60) ColumnarToRow [codegen id : 1] -Input [2]: [d_week_seq#82, d_year#83] +Input [2]: [d_week_seq#86, d_year#87] (61) Filter [codegen id : 1] -Input [2]: [d_week_seq#82, d_year#83] -Condition : ((isnotnull(d_year#83) AND (d_year#83 = 2002)) AND isnotnull(d_week_seq#82)) +Input [2]: [d_week_seq#86, d_year#87] +Condition : ((isnotnull(d_year#87) AND (d_year#87 = 2002)) AND isnotnull(d_week_seq#86)) (62) Project [codegen id : 1] -Output [1]: [d_week_seq#82] -Input [2]: [d_week_seq#82, d_year#83] +Output [1]: [d_week_seq#86] +Input [2]: [d_week_seq#86, d_year#87] (63) ObjectHashAggregate -Input [1]: [d_week_seq#82] +Input [1]: [d_week_seq#86] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#82, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [buf#103] -Results [1]: [buf#104] +Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#86, 42), 362, 9656, 0, 0)] +Aggregate Attributes [1]: [buf#107] +Results [1]: [buf#108] (64) Exchange -Input [1]: [buf#104] +Input [1]: [buf#108] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] (65) ObjectHashAggregate -Input [1]: [buf#104] +Input [1]: [buf#108] Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#82, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#82, 42), 362, 9656, 0, 0)#105] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#82, 42), 362, 9656, 0, 0)#105 AS bloomFilter#106] +Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#86, 42), 362, 9656, 0, 0)] +Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#86, 42), 362, 9656, 0, 0)#109] +Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#86, 42), 362, 9656, 0, 0)#109 AS bloomFilter#110] 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 3b66fa7c324b..8587ac85ae8c 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 @@ -152,61 +152,61 @@ Output [8]: [d_week_seq#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum# (25) HashAggregate [codegen id : 11] Input [8]: [d_week_seq#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57] Keys [1]: [d_week_seq#50] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#4 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#4 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#4 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#4 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#4 END))#32] -Results [8]: [d_week_seq#50, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#4 END))#26,17,2) AS sun_sales#59, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#4 END))#27,17,2) AS mon_sales#60, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#4 END))#28,17,2) AS tue_sales#61, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#4 END))#29,17,2) AS wed_sales#62, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#4 END))#30,17,2) AS thu_sales#63, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#4 END))#31,17,2) AS fri_sales#64, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#4 END))#32,17,2) AS sat_sales#65] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#59 END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#59 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#59 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#59 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#59 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#59 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#59 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#59 END))#32] +Results [8]: [d_week_seq#50, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#59 END))#26,17,2) AS sun_sales#60, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#59 END))#27,17,2) AS mon_sales#61, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#59 END))#28,17,2) AS tue_sales#62, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#59 END))#29,17,2) AS wed_sales#63, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#59 END))#30,17,2) AS thu_sales#64, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#59 END))#31,17,2) AS fri_sales#65, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#59 END))#32,17,2) AS sat_sales#66] (26) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#66, d_year#67] +Output [2]: [d_week_seq#67, d_year#68] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [d_week_seq#66, d_year#67] +Input [2]: [d_week_seq#67, d_year#68] (28) Filter [codegen id : 10] -Input [2]: [d_week_seq#66, d_year#67] -Condition : ((isnotnull(d_year#67) AND (d_year#67 = 2002)) AND isnotnull(d_week_seq#66)) +Input [2]: [d_week_seq#67, d_year#68] +Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) (29) Project [codegen id : 10] -Output [1]: [d_week_seq#66] -Input [2]: [d_week_seq#66, d_year#67] +Output [1]: [d_week_seq#67] +Input [2]: [d_week_seq#67, d_year#68] (30) BroadcastExchange -Input [1]: [d_week_seq#66] +Input [1]: [d_week_seq#67] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] (31) BroadcastHashJoin [codegen id : 11] Left keys [1]: [d_week_seq#50] -Right keys [1]: [d_week_seq#66] +Right keys [1]: [d_week_seq#67] Join type: Inner Join condition: None (32) Project [codegen id : 11] -Output [8]: [d_week_seq#50 AS d_week_seq2#68, sun_sales#59 AS sun_sales2#69, mon_sales#60 AS mon_sales2#70, tue_sales#61 AS tue_sales2#71, wed_sales#62 AS wed_sales2#72, thu_sales#63 AS thu_sales2#73, fri_sales#64 AS fri_sales2#74, sat_sales#65 AS sat_sales2#75] -Input [9]: [d_week_seq#50, sun_sales#59, mon_sales#60, tue_sales#61, wed_sales#62, thu_sales#63, fri_sales#64, sat_sales#65, d_week_seq#66] +Output [8]: [d_week_seq#50 AS d_week_seq2#69, sun_sales#60 AS sun_sales2#70, mon_sales#61 AS mon_sales2#71, tue_sales#62 AS tue_sales2#72, wed_sales#63 AS wed_sales2#73, thu_sales#64 AS thu_sales2#74, fri_sales#65 AS fri_sales2#75, sat_sales#66 AS sat_sales2#76] +Input [9]: [d_week_seq#50, sun_sales#60, mon_sales#61, tue_sales#62, wed_sales#63, thu_sales#64, fri_sales#65, sat_sales#66, d_week_seq#67] (33) BroadcastExchange -Input [8]: [d_week_seq2#68, sun_sales2#69, mon_sales2#70, tue_sales2#71, wed_sales2#72, thu_sales2#73, fri_sales2#74, sat_sales2#75] +Input [8]: [d_week_seq2#69, sun_sales2#70, mon_sales2#71, tue_sales2#72, wed_sales2#73, thu_sales2#74, fri_sales2#75, sat_sales2#76] Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=5] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#42] -Right keys [1]: [(d_week_seq2#68 - 53)] +Right keys [1]: [(d_week_seq2#69 - 53)] Join type: Inner Join condition: None (35) Project [codegen id : 12] -Output [8]: [d_week_seq1#42, round((sun_sales1#43 / sun_sales2#69), 2) AS round((sun_sales1 / sun_sales2), 2)#76, round((mon_sales1#44 / mon_sales2#70), 2) AS round((mon_sales1 / mon_sales2), 2)#77, round((tue_sales1#45 / tue_sales2#71), 2) AS round((tue_sales1 / tue_sales2), 2)#78, round((wed_sales1#46 / wed_sales2#72), 2) AS round((wed_sales1 / wed_sales2), 2)#79, round((thu_sales1#47 / thu_sales2#73), 2) AS round((thu_sales1 / thu_sales2), 2)#80, round((fri_sales1#48 / fri_sales2#74), 2) AS round((fri_sales1 / fri_sales2), 2)#81, round((sat_sales1#49 / sat_sales2#75), 2) AS round((sat_sales1 / sat_sales2), 2)#82] -Input [16]: [d_week_seq1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#68, sun_sales2#69, mon_sales2#70, tue_sales2#71, wed_sales2#72, thu_sales2#73, fri_sales2#74, sat_sales2#75] +Output [8]: [d_week_seq1#42, round((sun_sales1#43 / sun_sales2#70), 2) AS round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1#44 / mon_sales2#71), 2) AS round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1#45 / tue_sales2#72), 2) AS round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1#46 / wed_sales2#73), 2) AS round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1#47 / thu_sales2#74), 2) AS round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1#48 / fri_sales2#75), 2) AS round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1#49 / sat_sales2#76), 2) AS round((sat_sales1 / sat_sales2), 2)#83] +Input [16]: [d_week_seq1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#69, sun_sales2#70, mon_sales2#71, tue_sales2#72, wed_sales2#73, thu_sales2#74, fri_sales2#75, sat_sales2#76] (36) Exchange -Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#76, round((mon_sales1 / mon_sales2), 2)#77, round((tue_sales1 / tue_sales2), 2)#78, round((wed_sales1 / wed_sales2), 2)#79, round((thu_sales1 / thu_sales2), 2)#80, round((fri_sales1 / fri_sales2), 2)#81, round((sat_sales1 / sat_sales2), 2)#82] +Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1 / sat_sales2), 2)#83] Arguments: rangepartitioning(d_week_seq1#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] (37) Sort [codegen id : 13] -Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#76, round((mon_sales1 / mon_sales2), 2)#77, round((tue_sales1 / tue_sales2), 2)#78, round((wed_sales1 / wed_sales2), 2)#79, round((thu_sales1 / thu_sales2), 2)#80, round((fri_sales1 / fri_sales2), 2)#81, round((sat_sales1 / sat_sales2), 2)#82] +Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1 / sat_sales2), 2)#83] Arguments: [d_week_seq1#42 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt index 0cf3752714a5..0f44ef66d6aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt @@ -367,38 +367,38 @@ Join type: Inner Join condition: None (61) Project [codegen id : 25] -Output [3]: [d_date#41, i_item_sk#42, substr(i_item_desc#43, 1, 30) AS _groupingexpression#14] +Output [3]: [d_date#41, i_item_sk#42, substr(i_item_desc#43, 1, 30) AS _groupingexpression#44] Input [4]: [ss_item_sk#40, d_date#41, i_item_sk#42, i_item_desc#43] (62) HashAggregate [codegen id : 25] -Input [3]: [d_date#41, i_item_sk#42, _groupingexpression#14] -Keys [3]: [_groupingexpression#14, i_item_sk#42, d_date#41] +Input [3]: [d_date#41, i_item_sk#42, _groupingexpression#44] +Keys [3]: [_groupingexpression#44, i_item_sk#42, d_date#41] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#15] -Results [4]: [_groupingexpression#14, i_item_sk#42, d_date#41, count#16] +Results [4]: [_groupingexpression#44, i_item_sk#42, d_date#41, count#16] (63) HashAggregate [codegen id : 25] -Input [4]: [_groupingexpression#14, i_item_sk#42, d_date#41, count#16] -Keys [3]: [_groupingexpression#14, i_item_sk#42, d_date#41] +Input [4]: [_groupingexpression#44, i_item_sk#42, d_date#41, count#16] +Keys [3]: [_groupingexpression#44, i_item_sk#42, d_date#41] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#42 AS item_sk#44, count(1)#17 AS cnt#45] +Results [2]: [i_item_sk#42 AS item_sk#45, count(1)#17 AS cnt#46] (64) Filter [codegen id : 25] -Input [2]: [item_sk#44, cnt#45] -Condition : (cnt#45 > 4) +Input [2]: [item_sk#45, cnt#46] +Condition : (cnt#46 > 4) (65) Project [codegen id : 25] -Output [1]: [item_sk#44] -Input [2]: [item_sk#44, cnt#45] +Output [1]: [item_sk#45] +Input [2]: [item_sk#45, cnt#46] (66) Sort [codegen id : 25] -Input [1]: [item_sk#44] -Arguments: [item_sk#44 ASC NULLS FIRST], false, 0 +Input [1]: [item_sk#45] +Arguments: [item_sk#45 ASC NULLS FIRST], false, 0 (67) SortMergeJoin [codegen id : 26] Left keys [1]: [ws_item_sk#35] -Right keys [1]: [item_sk#44] +Right keys [1]: [item_sk#45] Join type: LeftSemi Join condition: None @@ -415,58 +415,58 @@ Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_da Arguments: [ws_bill_customer_sk#36 ASC NULLS FIRST], false, 0 (71) ReusedExchange [Reuses operator id: 33] -Output [3]: [ss_customer_sk#46, ss_quantity#47, ss_sales_price#48] +Output [3]: [ss_customer_sk#47, ss_quantity#48, ss_sales_price#49] (72) Sort [codegen id : 29] -Input [3]: [ss_customer_sk#46, ss_quantity#47, ss_sales_price#48] -Arguments: [ss_customer_sk#46 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#47, ss_quantity#48, ss_sales_price#49] +Arguments: [ss_customer_sk#47 ASC NULLS FIRST], false, 0 (73) ReusedExchange [Reuses operator id: 38] -Output [1]: [c_customer_sk#49] +Output [1]: [c_customer_sk#50] (74) Sort [codegen id : 31] -Input [1]: [c_customer_sk#49] -Arguments: [c_customer_sk#49 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#50] +Arguments: [c_customer_sk#50 ASC NULLS FIRST], false, 0 (75) SortMergeJoin [codegen id : 32] -Left keys [1]: [ss_customer_sk#46] -Right keys [1]: [c_customer_sk#49] +Left keys [1]: [ss_customer_sk#47] +Right keys [1]: [c_customer_sk#50] Join type: Inner Join condition: None (76) Project [codegen id : 32] -Output [3]: [ss_quantity#47, ss_sales_price#48, c_customer_sk#49] -Input [4]: [ss_customer_sk#46, ss_quantity#47, ss_sales_price#48, c_customer_sk#49] +Output [3]: [ss_quantity#48, ss_sales_price#49, c_customer_sk#50] +Input [4]: [ss_customer_sk#47, ss_quantity#48, ss_sales_price#49, c_customer_sk#50] (77) HashAggregate [codegen id : 32] -Input [3]: [ss_quantity#47, ss_sales_price#48, c_customer_sk#49] -Keys [1]: [c_customer_sk#49] -Functions [1]: [partial_sum((cast(ss_quantity#47 as decimal(10,0)) * ss_sales_price#48))] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [3]: [c_customer_sk#49, sum#52, isEmpty#53] +Input [3]: [ss_quantity#48, ss_sales_price#49, c_customer_sk#50] +Keys [1]: [c_customer_sk#50] +Functions [1]: [partial_sum((cast(ss_quantity#48 as decimal(10,0)) * ss_sales_price#49))] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [3]: [c_customer_sk#50, sum#53, isEmpty#54] (78) HashAggregate [codegen id : 32] -Input [3]: [c_customer_sk#49, sum#52, isEmpty#53] -Keys [1]: [c_customer_sk#49] -Functions [1]: [sum((cast(ss_quantity#47 as decimal(10,0)) * ss_sales_price#48))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#47 as decimal(10,0)) * ss_sales_price#48))#29] -Results [2]: [c_customer_sk#49, sum((cast(ss_quantity#47 as decimal(10,0)) * ss_sales_price#48))#29 AS ssales#54] +Input [3]: [c_customer_sk#50, sum#53, isEmpty#54] +Keys [1]: [c_customer_sk#50] +Functions [1]: [sum((cast(ss_quantity#48 as decimal(10,0)) * ss_sales_price#49))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#48 as decimal(10,0)) * ss_sales_price#49))#29] +Results [2]: [c_customer_sk#50, sum((cast(ss_quantity#48 as decimal(10,0)) * ss_sales_price#49))#29 AS ssales#55] (79) Filter [codegen id : 32] -Input [2]: [c_customer_sk#49, ssales#54] -Condition : (isnotnull(ssales#54) AND (cast(ssales#54 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) +Input [2]: [c_customer_sk#50, ssales#55] +Condition : (isnotnull(ssales#55) AND (cast(ssales#55 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) (80) Project [codegen id : 32] -Output [1]: [c_customer_sk#49] -Input [2]: [c_customer_sk#49, ssales#54] +Output [1]: [c_customer_sk#50] +Input [2]: [c_customer_sk#50, ssales#55] (81) Sort [codegen id : 32] -Input [1]: [c_customer_sk#49] -Arguments: [c_customer_sk#49 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#50] +Arguments: [c_customer_sk#50 ASC NULLS FIRST], false, 0 (82) SortMergeJoin [codegen id : 34] Left keys [1]: [ws_bill_customer_sk#36] -Right keys [1]: [c_customer_sk#49] +Right keys [1]: [c_customer_sk#50] Join type: LeftSemi Join condition: None @@ -475,17 +475,17 @@ Output [3]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] (84) ReusedExchange [Reuses operator id: 95] -Output [1]: [d_date_sk#55] +Output [1]: [d_date_sk#56] (85) BroadcastHashJoin [codegen id : 34] Left keys [1]: [ws_sold_date_sk#39] -Right keys [1]: [d_date_sk#55] +Right keys [1]: [d_date_sk#56] Join type: Inner Join condition: None (86) Project [codegen id : 34] -Output [1]: [(cast(ws_quantity#37 as decimal(10,0)) * ws_list_price#38) AS sales#56] -Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#55] +Output [1]: [(cast(ws_quantity#37 as decimal(10,0)) * ws_list_price#38) AS sales#57] +Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#56] (87) Union @@ -493,19 +493,19 @@ Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#55] Input [1]: [sales#34] Keys: [] Functions [1]: [partial_sum(sales#34)] -Aggregate Attributes [2]: [sum#57, isEmpty#58] -Results [2]: [sum#59, isEmpty#60] +Aggregate Attributes [2]: [sum#58, isEmpty#59] +Results [2]: [sum#60, isEmpty#61] (89) Exchange -Input [2]: [sum#59, isEmpty#60] +Input [2]: [sum#60, isEmpty#61] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] (90) HashAggregate [codegen id : 36] -Input [2]: [sum#59, isEmpty#60] +Input [2]: [sum#60, isEmpty#61] Keys: [] Functions [1]: [sum(sales#34)] -Aggregate Attributes [1]: [sum(sales#34)#61] -Results [1]: [sum(sales#34)#61 AS sum(sales)#62] +Aggregate Attributes [1]: [sum(sales#34)#62] +Results [1]: [sum(sales#34)#62 AS sum(sales)#63] ===== Subqueries ===== @@ -518,22 +518,22 @@ BroadcastExchange (95) (91) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#33, d_year#63, d_moy#64] +Output [3]: [d_date_sk#33, d_year#64, d_moy#65] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#33, d_year#63, d_moy#64] +Input [3]: [d_date_sk#33, d_year#64, d_moy#65] (93) Filter [codegen id : 1] -Input [3]: [d_date_sk#33, d_year#63, d_moy#64] -Condition : ((((isnotnull(d_year#63) AND isnotnull(d_moy#64)) AND (d_year#63 = 2000)) AND (d_moy#64 = 2)) AND isnotnull(d_date_sk#33)) +Input [3]: [d_date_sk#33, d_year#64, d_moy#65] +Condition : ((((isnotnull(d_year#64) AND isnotnull(d_moy#65)) AND (d_year#64 = 2000)) AND (d_moy#65 = 2)) AND isnotnull(d_date_sk#33)) (94) Project [codegen id : 1] Output [1]: [d_date_sk#33] -Input [3]: [d_date_sk#33, d_year#63, d_moy#64] +Input [3]: [d_date_sk#33, d_year#64, d_moy#65] (95) BroadcastExchange Input [1]: [d_date_sk#33] @@ -548,22 +548,22 @@ BroadcastExchange (100) (96) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#65] +Output [3]: [d_date_sk#10, d_date#11, d_year#66] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (97) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#10, d_date#11, d_year#65] +Input [3]: [d_date_sk#10, d_date#11, d_year#66] (98) Filter [codegen id : 1] -Input [3]: [d_date_sk#10, d_date#11, d_year#65] -Condition : (d_year#65 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +Input [3]: [d_date_sk#10, d_date#11, d_year#66] +Condition : (d_year#66 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) (99) Project [codegen id : 1] Output [2]: [d_date_sk#10, d_date#11] -Input [3]: [d_date_sk#10, d_date#11, d_year#65] +Input [3]: [d_date_sk#10, d_date#11, d_year#66] (100) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] @@ -590,91 +590,91 @@ Subquery:3 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquer (101) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69] +Output [4]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#69), dynamicpruningexpression(ss_sold_date_sk#69 IN dynamicpruning#70)] +PartitionFilters: [isnotnull(ss_sold_date_sk#70), dynamicpruningexpression(ss_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69] +Input [4]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70] (103) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69] -Condition : isnotnull(ss_customer_sk#66) +Input [4]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70] +Condition : isnotnull(ss_customer_sk#67) (104) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#71] +Output [1]: [d_date_sk#72] (105) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#69] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ss_sold_date_sk#70] +Right keys [1]: [d_date_sk#72] Join type: Inner Join condition: None (106) Project [codegen id : 2] -Output [3]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68] -Input [5]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69, d_date_sk#71] +Output [3]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69] +Input [5]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70, d_date_sk#72] (107) Exchange -Input [3]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68] -Arguments: hashpartitioning(ss_customer_sk#66, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Input [3]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69] +Arguments: hashpartitioning(ss_customer_sk#67, 5), ENSURE_REQUIREMENTS, [plan_id=12] (108) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68] -Arguments: [ss_customer_sk#66 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69] +Arguments: [ss_customer_sk#67 ASC NULLS FIRST], false, 0 (109) ReusedExchange [Reuses operator id: 38] -Output [1]: [c_customer_sk#72] +Output [1]: [c_customer_sk#73] (110) Sort [codegen id : 5] -Input [1]: [c_customer_sk#72] -Arguments: [c_customer_sk#72 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#73] +Arguments: [c_customer_sk#73 ASC NULLS FIRST], false, 0 (111) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#66] -Right keys [1]: [c_customer_sk#72] +Left keys [1]: [ss_customer_sk#67] +Right keys [1]: [c_customer_sk#73] Join type: Inner Join condition: None (112) Project [codegen id : 6] -Output [3]: [ss_quantity#67, ss_sales_price#68, c_customer_sk#72] -Input [4]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, c_customer_sk#72] +Output [3]: [ss_quantity#68, ss_sales_price#69, c_customer_sk#73] +Input [4]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, c_customer_sk#73] (113) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#67, ss_sales_price#68, c_customer_sk#72] -Keys [1]: [c_customer_sk#72] -Functions [1]: [partial_sum((cast(ss_quantity#67 as decimal(10,0)) * ss_sales_price#68))] -Aggregate Attributes [2]: [sum#73, isEmpty#74] -Results [3]: [c_customer_sk#72, sum#75, isEmpty#76] +Input [3]: [ss_quantity#68, ss_sales_price#69, c_customer_sk#73] +Keys [1]: [c_customer_sk#73] +Functions [1]: [partial_sum((cast(ss_quantity#68 as decimal(10,0)) * ss_sales_price#69))] +Aggregate Attributes [2]: [sum#74, isEmpty#75] +Results [3]: [c_customer_sk#73, sum#76, isEmpty#77] (114) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#72, sum#75, isEmpty#76] -Keys [1]: [c_customer_sk#72] -Functions [1]: [sum((cast(ss_quantity#67 as decimal(10,0)) * ss_sales_price#68))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#67 as decimal(10,0)) * ss_sales_price#68))#77] -Results [1]: [sum((cast(ss_quantity#67 as decimal(10,0)) * ss_sales_price#68))#77 AS csales#78] +Input [3]: [c_customer_sk#73, sum#76, isEmpty#77] +Keys [1]: [c_customer_sk#73] +Functions [1]: [sum((cast(ss_quantity#68 as decimal(10,0)) * ss_sales_price#69))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#68 as decimal(10,0)) * ss_sales_price#69))#78] +Results [1]: [sum((cast(ss_quantity#68 as decimal(10,0)) * ss_sales_price#69))#78 AS csales#79] (115) HashAggregate [codegen id : 6] -Input [1]: [csales#78] +Input [1]: [csales#79] Keys: [] -Functions [1]: [partial_max(csales#78)] -Aggregate Attributes [1]: [max#79] -Results [1]: [max#80] +Functions [1]: [partial_max(csales#79)] +Aggregate Attributes [1]: [max#80] +Results [1]: [max#81] (116) Exchange -Input [1]: [max#80] +Input [1]: [max#81] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] (117) HashAggregate [codegen id : 7] -Input [1]: [max#80] +Input [1]: [max#81] Keys: [] -Functions [1]: [max(csales#78)] -Aggregate Attributes [1]: [max(csales#78)#81] -Results [1]: [max(csales#78)#81 AS tpcds_cmax#82] +Functions [1]: [max(csales#79)] +Aggregate Attributes [1]: [max(csales#79)#82] +Results [1]: [max(csales#79)#82 AS tpcds_cmax#83] -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#69 IN dynamicpruning#70 +Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (122) +- * Project (121) +- * Filter (120) @@ -683,25 +683,25 @@ BroadcastExchange (122) (118) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#71, d_year#83] +Output [2]: [d_date_sk#72, d_year#84] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (119) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#71, d_year#83] +Input [2]: [d_date_sk#72, d_year#84] (120) Filter [codegen id : 1] -Input [2]: [d_date_sk#71, d_year#83] -Condition : (d_year#83 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#71)) +Input [2]: [d_date_sk#72, d_year#84] +Condition : (d_year#84 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#72)) (121) Project [codegen id : 1] -Output [1]: [d_date_sk#71] -Input [2]: [d_date_sk#71, d_year#83] +Output [1]: [d_date_sk#72] +Input [2]: [d_date_sk#72, d_year#84] (122) BroadcastExchange -Input [1]: [d_date_sk#71] +Input [1]: [d_date_sk#72] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt index 038f9478f173..7b8d3a1d10c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt @@ -518,38 +518,38 @@ Join type: Inner Join condition: None (84) Project [codegen id : 35] -Output [3]: [d_date#49, i_item_sk#50, substr(i_item_desc#51, 1, 30) AS _groupingexpression#14] +Output [3]: [d_date#49, i_item_sk#50, substr(i_item_desc#51, 1, 30) AS _groupingexpression#52] Input [4]: [ss_item_sk#48, d_date#49, i_item_sk#50, i_item_desc#51] (85) HashAggregate [codegen id : 35] -Input [3]: [d_date#49, i_item_sk#50, _groupingexpression#14] -Keys [3]: [_groupingexpression#14, i_item_sk#50, d_date#49] +Input [3]: [d_date#49, i_item_sk#50, _groupingexpression#52] +Keys [3]: [_groupingexpression#52, i_item_sk#50, d_date#49] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#15] -Results [4]: [_groupingexpression#14, i_item_sk#50, d_date#49, count#16] +Results [4]: [_groupingexpression#52, i_item_sk#50, d_date#49, count#16] (86) HashAggregate [codegen id : 35] -Input [4]: [_groupingexpression#14, i_item_sk#50, d_date#49, count#16] -Keys [3]: [_groupingexpression#14, i_item_sk#50, d_date#49] +Input [4]: [_groupingexpression#52, i_item_sk#50, d_date#49, count#16] +Keys [3]: [_groupingexpression#52, i_item_sk#50, d_date#49] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#50 AS item_sk#52, count(1)#17 AS cnt#53] +Results [2]: [i_item_sk#50 AS item_sk#53, count(1)#17 AS cnt#54] (87) Filter [codegen id : 35] -Input [2]: [item_sk#52, cnt#53] -Condition : (cnt#53 > 4) +Input [2]: [item_sk#53, cnt#54] +Condition : (cnt#54 > 4) (88) Project [codegen id : 35] -Output [1]: [item_sk#52] -Input [2]: [item_sk#52, cnt#53] +Output [1]: [item_sk#53] +Input [2]: [item_sk#53, cnt#54] (89) Sort [codegen id : 35] -Input [1]: [item_sk#52] -Arguments: [item_sk#52 ASC NULLS FIRST], false, 0 +Input [1]: [item_sk#53] +Arguments: [item_sk#53 ASC NULLS FIRST], false, 0 (90) SortMergeJoin [codegen id : 36] Left keys [1]: [ws_item_sk#43] -Right keys [1]: [item_sk#52] +Right keys [1]: [item_sk#53] Join type: LeftSemi Join condition: None @@ -566,164 +566,164 @@ Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_da Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 (94) ReusedExchange [Reuses operator id: 34] -Output [3]: [ss_customer_sk#54, ss_quantity#55, ss_sales_price#56] +Output [3]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57] (95) Sort [codegen id : 39] -Input [3]: [ss_customer_sk#54, ss_quantity#55, ss_sales_price#56] -Arguments: [ss_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57] +Arguments: [ss_customer_sk#55 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: 39] -Output [1]: [c_customer_sk#57] +Output [1]: [c_customer_sk#58] (97) Sort [codegen id : 41] -Input [1]: [c_customer_sk#57] -Arguments: [c_customer_sk#57 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#58] +Arguments: [c_customer_sk#58 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 42] -Left keys [1]: [ss_customer_sk#54] -Right keys [1]: [c_customer_sk#57] +Left keys [1]: [ss_customer_sk#55] +Right keys [1]: [c_customer_sk#58] Join type: Inner Join condition: None (99) Project [codegen id : 42] -Output [3]: [ss_quantity#55, ss_sales_price#56, c_customer_sk#57] -Input [4]: [ss_customer_sk#54, ss_quantity#55, ss_sales_price#56, c_customer_sk#57] +Output [3]: [ss_quantity#56, ss_sales_price#57, c_customer_sk#58] +Input [4]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57, c_customer_sk#58] (100) HashAggregate [codegen id : 42] -Input [3]: [ss_quantity#55, ss_sales_price#56, c_customer_sk#57] -Keys [1]: [c_customer_sk#57] -Functions [1]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))] -Aggregate Attributes [2]: [sum#58, isEmpty#59] -Results [3]: [c_customer_sk#57, sum#60, isEmpty#61] +Input [3]: [ss_quantity#56, ss_sales_price#57, c_customer_sk#58] +Keys [1]: [c_customer_sk#58] +Functions [1]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_sales_price#57))] +Aggregate Attributes [2]: [sum#59, isEmpty#60] +Results [3]: [c_customer_sk#58, sum#61, isEmpty#62] (101) HashAggregate [codegen id : 42] -Input [3]: [c_customer_sk#57, sum#60, isEmpty#61] -Keys [1]: [c_customer_sk#57] -Functions [1]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))#29] -Results [2]: [c_customer_sk#57, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))#29 AS ssales#62] +Input [3]: [c_customer_sk#58, sum#61, isEmpty#62] +Keys [1]: [c_customer_sk#58] +Functions [1]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_sales_price#57))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_sales_price#57))#29] +Results [2]: [c_customer_sk#58, sum((cast(ss_quantity#56 as decimal(10,0)) * ss_sales_price#57))#29 AS ssales#63] (102) Filter [codegen id : 42] -Input [2]: [c_customer_sk#57, ssales#62] -Condition : (isnotnull(ssales#62) AND (cast(ssales#62 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) +Input [2]: [c_customer_sk#58, ssales#63] +Condition : (isnotnull(ssales#63) AND (cast(ssales#63 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) (103) Project [codegen id : 42] -Output [1]: [c_customer_sk#57] -Input [2]: [c_customer_sk#57, ssales#62] +Output [1]: [c_customer_sk#58] +Input [2]: [c_customer_sk#58, ssales#63] (104) Sort [codegen id : 42] -Input [1]: [c_customer_sk#57] -Arguments: [c_customer_sk#57 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#58] +Arguments: [c_customer_sk#58 ASC NULLS FIRST], false, 0 (105) SortMergeJoin [codegen id : 44] Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#57] +Right keys [1]: [c_customer_sk#58] Join type: LeftSemi Join condition: None (106) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#63] +Output [1]: [d_date_sk#64] (107) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_sold_date_sk#47] -Right keys [1]: [d_date_sk#63] +Right keys [1]: [d_date_sk#64] Join type: Inner Join condition: None (108) Project [codegen id : 44] Output [3]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] -Input [5]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#63] +Input [5]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#64] (109) ReusedExchange [Reuses operator id: 55] -Output [3]: [c_customer_sk#64, c_first_name#65, c_last_name#66] +Output [3]: [c_customer_sk#65, c_first_name#66, c_last_name#67] (110) Sort [codegen id : 46] -Input [3]: [c_customer_sk#64, c_first_name#65, c_last_name#66] -Arguments: [c_customer_sk#64 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#65, c_first_name#66, c_last_name#67] +Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 (111) ReusedExchange [Reuses operator id: 34] -Output [3]: [ss_customer_sk#54, ss_quantity#55, ss_sales_price#56] +Output [3]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57] (112) Sort [codegen id : 48] -Input [3]: [ss_customer_sk#54, ss_quantity#55, ss_sales_price#56] -Arguments: [ss_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57] +Arguments: [ss_customer_sk#55 ASC NULLS FIRST], false, 0 (113) ReusedExchange [Reuses operator id: 39] -Output [1]: [c_customer_sk#57] +Output [1]: [c_customer_sk#58] (114) Sort [codegen id : 50] -Input [1]: [c_customer_sk#57] -Arguments: [c_customer_sk#57 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#58] +Arguments: [c_customer_sk#58 ASC NULLS FIRST], false, 0 (115) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#54] -Right keys [1]: [c_customer_sk#57] +Left keys [1]: [ss_customer_sk#55] +Right keys [1]: [c_customer_sk#58] Join type: Inner Join condition: None (116) Project [codegen id : 51] -Output [3]: [ss_quantity#55, ss_sales_price#56, c_customer_sk#57] -Input [4]: [ss_customer_sk#54, ss_quantity#55, ss_sales_price#56, c_customer_sk#57] +Output [3]: [ss_quantity#56, ss_sales_price#57, c_customer_sk#58] +Input [4]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57, c_customer_sk#58] (117) HashAggregate [codegen id : 51] -Input [3]: [ss_quantity#55, ss_sales_price#56, c_customer_sk#57] -Keys [1]: [c_customer_sk#57] -Functions [1]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))] -Aggregate Attributes [2]: [sum#58, isEmpty#59] -Results [3]: [c_customer_sk#57, sum#60, isEmpty#61] +Input [3]: [ss_quantity#56, ss_sales_price#57, c_customer_sk#58] +Keys [1]: [c_customer_sk#58] +Functions [1]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_sales_price#57))] +Aggregate Attributes [2]: [sum#59, isEmpty#60] +Results [3]: [c_customer_sk#58, sum#61, isEmpty#62] (118) HashAggregate [codegen id : 51] -Input [3]: [c_customer_sk#57, sum#60, isEmpty#61] -Keys [1]: [c_customer_sk#57] -Functions [1]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))#29] -Results [2]: [c_customer_sk#57, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))#29 AS ssales#62] +Input [3]: [c_customer_sk#58, sum#61, isEmpty#62] +Keys [1]: [c_customer_sk#58] +Functions [1]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_sales_price#57))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_sales_price#57))#29] +Results [2]: [c_customer_sk#58, sum((cast(ss_quantity#56 as decimal(10,0)) * ss_sales_price#57))#29 AS ssales#63] (119) Filter [codegen id : 51] -Input [2]: [c_customer_sk#57, ssales#62] -Condition : (isnotnull(ssales#62) AND (cast(ssales#62 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) +Input [2]: [c_customer_sk#58, ssales#63] +Condition : (isnotnull(ssales#63) AND (cast(ssales#63 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) (120) Project [codegen id : 51] -Output [1]: [c_customer_sk#57] -Input [2]: [c_customer_sk#57, ssales#62] +Output [1]: [c_customer_sk#58] +Input [2]: [c_customer_sk#58, ssales#63] (121) Sort [codegen id : 51] -Input [1]: [c_customer_sk#57] -Arguments: [c_customer_sk#57 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#58] +Arguments: [c_customer_sk#58 ASC NULLS FIRST], false, 0 (122) SortMergeJoin [codegen id : 52] -Left keys [1]: [c_customer_sk#64] -Right keys [1]: [c_customer_sk#57] +Left keys [1]: [c_customer_sk#65] +Right keys [1]: [c_customer_sk#58] Join type: LeftSemi Join condition: None (123) SortMergeJoin [codegen id : 53] Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#64] +Right keys [1]: [c_customer_sk#65] Join type: Inner Join condition: None (124) Project [codegen id : 53] -Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#65, c_last_name#66] -Input [6]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, c_customer_sk#64, c_first_name#65, c_last_name#66] +Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#66, c_last_name#67] +Input [6]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, c_customer_sk#65, c_first_name#66, c_last_name#67] (125) HashAggregate [codegen id : 53] -Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#65, c_last_name#66] -Keys [2]: [c_last_name#66, c_first_name#65] +Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#66, c_last_name#67] +Keys [2]: [c_last_name#67, c_first_name#66] Functions [1]: [partial_sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] -Aggregate Attributes [2]: [sum#67, isEmpty#68] -Results [4]: [c_last_name#66, c_first_name#65, sum#69, isEmpty#70] +Aggregate Attributes [2]: [sum#68, isEmpty#69] +Results [4]: [c_last_name#67, c_first_name#66, sum#70, isEmpty#71] (126) Exchange -Input [4]: [c_last_name#66, c_first_name#65, sum#69, isEmpty#70] -Arguments: hashpartitioning(c_last_name#66, c_first_name#65, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Input [4]: [c_last_name#67, c_first_name#66, sum#70, isEmpty#71] +Arguments: hashpartitioning(c_last_name#67, c_first_name#66, 5), ENSURE_REQUIREMENTS, [plan_id=11] (127) HashAggregate [codegen id : 54] -Input [4]: [c_last_name#66, c_first_name#65, sum#69, isEmpty#70] -Keys [2]: [c_last_name#66, c_first_name#65] +Input [4]: [c_last_name#67, c_first_name#66, sum#70, isEmpty#71] +Keys [2]: [c_last_name#67, c_first_name#66] Functions [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] -Aggregate Attributes [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#71] -Results [3]: [c_last_name#66, c_first_name#65, sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#71 AS sales#72] +Aggregate Attributes [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#72] +Results [3]: [c_last_name#67, c_first_name#66, sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#72 AS sales#73] (128) Union @@ -742,22 +742,22 @@ BroadcastExchange (134) (130) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#33, d_year#73, d_moy#74] +Output [3]: [d_date_sk#33, d_year#74, d_moy#75] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (131) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#33, d_year#73, d_moy#74] +Input [3]: [d_date_sk#33, d_year#74, d_moy#75] (132) Filter [codegen id : 1] -Input [3]: [d_date_sk#33, d_year#73, d_moy#74] -Condition : ((((isnotnull(d_year#73) AND isnotnull(d_moy#74)) AND (d_year#73 = 2000)) AND (d_moy#74 = 2)) AND isnotnull(d_date_sk#33)) +Input [3]: [d_date_sk#33, d_year#74, d_moy#75] +Condition : ((((isnotnull(d_year#74) AND isnotnull(d_moy#75)) AND (d_year#74 = 2000)) AND (d_moy#75 = 2)) AND isnotnull(d_date_sk#33)) (133) Project [codegen id : 1] Output [1]: [d_date_sk#33] -Input [3]: [d_date_sk#33, d_year#73, d_moy#74] +Input [3]: [d_date_sk#33, d_year#74, d_moy#75] (134) BroadcastExchange Input [1]: [d_date_sk#33] @@ -772,22 +772,22 @@ BroadcastExchange (139) (135) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#75] +Output [3]: [d_date_sk#10, d_date#11, d_year#76] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (136) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#10, d_date#11, d_year#75] +Input [3]: [d_date_sk#10, d_date#11, d_year#76] (137) Filter [codegen id : 1] -Input [3]: [d_date_sk#10, d_date#11, d_year#75] -Condition : (d_year#75 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +Input [3]: [d_date_sk#10, d_date#11, d_year#76] +Condition : (d_year#76 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) (138) Project [codegen id : 1] Output [2]: [d_date_sk#10, d_date#11] -Input [3]: [d_date_sk#10, d_date#11, d_year#75] +Input [3]: [d_date_sk#10, d_date#11, d_year#76] (139) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] @@ -814,91 +814,91 @@ Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquer (140) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#76, ss_quantity#77, ss_sales_price#78, ss_sold_date_sk#79] +Output [4]: [ss_customer_sk#77, ss_quantity#78, ss_sales_price#79, ss_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#79), dynamicpruningexpression(ss_sold_date_sk#79 IN dynamicpruning#80)] +PartitionFilters: [isnotnull(ss_sold_date_sk#80), dynamicpruningexpression(ss_sold_date_sk#80 IN dynamicpruning#81)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (141) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#76, ss_quantity#77, ss_sales_price#78, ss_sold_date_sk#79] +Input [4]: [ss_customer_sk#77, ss_quantity#78, ss_sales_price#79, ss_sold_date_sk#80] (142) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#76, ss_quantity#77, ss_sales_price#78, ss_sold_date_sk#79] -Condition : isnotnull(ss_customer_sk#76) +Input [4]: [ss_customer_sk#77, ss_quantity#78, ss_sales_price#79, ss_sold_date_sk#80] +Condition : isnotnull(ss_customer_sk#77) (143) ReusedExchange [Reuses operator id: 161] -Output [1]: [d_date_sk#81] +Output [1]: [d_date_sk#82] (144) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#79] -Right keys [1]: [d_date_sk#81] +Left keys [1]: [ss_sold_date_sk#80] +Right keys [1]: [d_date_sk#82] Join type: Inner Join condition: None (145) Project [codegen id : 2] -Output [3]: [ss_customer_sk#76, ss_quantity#77, ss_sales_price#78] -Input [5]: [ss_customer_sk#76, ss_quantity#77, ss_sales_price#78, ss_sold_date_sk#79, d_date_sk#81] +Output [3]: [ss_customer_sk#77, ss_quantity#78, ss_sales_price#79] +Input [5]: [ss_customer_sk#77, ss_quantity#78, ss_sales_price#79, ss_sold_date_sk#80, d_date_sk#82] (146) Exchange -Input [3]: [ss_customer_sk#76, ss_quantity#77, ss_sales_price#78] -Arguments: hashpartitioning(ss_customer_sk#76, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Input [3]: [ss_customer_sk#77, ss_quantity#78, ss_sales_price#79] +Arguments: hashpartitioning(ss_customer_sk#77, 5), ENSURE_REQUIREMENTS, [plan_id=14] (147) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#76, ss_quantity#77, ss_sales_price#78] -Arguments: [ss_customer_sk#76 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#77, ss_quantity#78, ss_sales_price#79] +Arguments: [ss_customer_sk#77 ASC NULLS FIRST], false, 0 (148) ReusedExchange [Reuses operator id: 39] -Output [1]: [c_customer_sk#82] +Output [1]: [c_customer_sk#83] (149) Sort [codegen id : 5] -Input [1]: [c_customer_sk#82] -Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#83] +Arguments: [c_customer_sk#83 ASC NULLS FIRST], false, 0 (150) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#76] -Right keys [1]: [c_customer_sk#82] +Left keys [1]: [ss_customer_sk#77] +Right keys [1]: [c_customer_sk#83] Join type: Inner Join condition: None (151) Project [codegen id : 6] -Output [3]: [ss_quantity#77, ss_sales_price#78, c_customer_sk#82] -Input [4]: [ss_customer_sk#76, ss_quantity#77, ss_sales_price#78, c_customer_sk#82] +Output [3]: [ss_quantity#78, ss_sales_price#79, c_customer_sk#83] +Input [4]: [ss_customer_sk#77, ss_quantity#78, ss_sales_price#79, c_customer_sk#83] (152) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#77, ss_sales_price#78, c_customer_sk#82] -Keys [1]: [c_customer_sk#82] -Functions [1]: [partial_sum((cast(ss_quantity#77 as decimal(10,0)) * ss_sales_price#78))] -Aggregate Attributes [2]: [sum#83, isEmpty#84] -Results [3]: [c_customer_sk#82, sum#85, isEmpty#86] +Input [3]: [ss_quantity#78, ss_sales_price#79, c_customer_sk#83] +Keys [1]: [c_customer_sk#83] +Functions [1]: [partial_sum((cast(ss_quantity#78 as decimal(10,0)) * ss_sales_price#79))] +Aggregate Attributes [2]: [sum#84, isEmpty#85] +Results [3]: [c_customer_sk#83, sum#86, isEmpty#87] (153) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#82, sum#85, isEmpty#86] -Keys [1]: [c_customer_sk#82] -Functions [1]: [sum((cast(ss_quantity#77 as decimal(10,0)) * ss_sales_price#78))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#77 as decimal(10,0)) * ss_sales_price#78))#87] -Results [1]: [sum((cast(ss_quantity#77 as decimal(10,0)) * ss_sales_price#78))#87 AS csales#88] +Input [3]: [c_customer_sk#83, sum#86, isEmpty#87] +Keys [1]: [c_customer_sk#83] +Functions [1]: [sum((cast(ss_quantity#78 as decimal(10,0)) * ss_sales_price#79))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#78 as decimal(10,0)) * ss_sales_price#79))#88] +Results [1]: [sum((cast(ss_quantity#78 as decimal(10,0)) * ss_sales_price#79))#88 AS csales#89] (154) HashAggregate [codegen id : 6] -Input [1]: [csales#88] +Input [1]: [csales#89] Keys: [] -Functions [1]: [partial_max(csales#88)] -Aggregate Attributes [1]: [max#89] -Results [1]: [max#90] +Functions [1]: [partial_max(csales#89)] +Aggregate Attributes [1]: [max#90] +Results [1]: [max#91] (155) Exchange -Input [1]: [max#90] +Input [1]: [max#91] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] (156) HashAggregate [codegen id : 7] -Input [1]: [max#90] +Input [1]: [max#91] Keys: [] -Functions [1]: [max(csales#88)] -Aggregate Attributes [1]: [max(csales#88)#91] -Results [1]: [max(csales#88)#91 AS tpcds_cmax#92] +Functions [1]: [max(csales#89)] +Aggregate Attributes [1]: [max(csales#89)#92] +Results [1]: [max(csales#89)#92 AS tpcds_cmax#93] -Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#79 IN dynamicpruning#80 +Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#81 BroadcastExchange (161) +- * Project (160) +- * Filter (159) @@ -907,25 +907,25 @@ BroadcastExchange (161) (157) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#81, d_year#93] +Output [2]: [d_date_sk#82, d_year#94] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (158) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#81, d_year#93] +Input [2]: [d_date_sk#82, d_year#94] (159) Filter [codegen id : 1] -Input [2]: [d_date_sk#81, d_year#93] -Condition : (d_year#93 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#81)) +Input [2]: [d_date_sk#82, d_year#94] +Condition : (d_year#94 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#82)) (160) Project [codegen id : 1] -Output [1]: [d_date_sk#81] -Input [2]: [d_date_sk#81, d_year#93] +Output [1]: [d_date_sk#82] +Input [2]: [d_date_sk#82, d_year#94] (161) BroadcastExchange -Input [1]: [d_date_sk#81] +Input [1]: [d_date_sk#82] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt index da3f54ac0a32..3f63e2e92df7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt @@ -236,22 +236,22 @@ Input [9]: [w_warehouse_name#35, w_warehouse_sk#34, i_item_sk#33, d_moy#32, n#41 Keys [4]: [w_warehouse_name#35, w_warehouse_sk#34, i_item_sk#33, d_moy#32] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] +Results [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#46, avg(inv_quantity_on_hand#28)#22 AS mean#47] (41) Filter [codegen id : 11] -Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stdev#23, mean#24] -Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END +Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stdev#46, mean#47] +Condition : CASE WHEN (mean#47 = 0.0) THEN false ELSE ((stdev#46 / mean#47) > 1.0) END (42) Project [codegen id : 11] -Output [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] -Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stdev#23, mean#24] +Output [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#47, CASE WHEN (mean#47 = 0.0) THEN null ELSE (stdev#46 / mean#47) END AS cov#48] +Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stdev#46, mean#47] (43) Exchange -Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#46, cov#47] +Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#47, cov#48] Arguments: hashpartitioning(i_item_sk#33, w_warehouse_sk#34, 5), ENSURE_REQUIREMENTS, [plan_id=6] (44) Sort [codegen id : 12] -Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#46, cov#47] +Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#47, cov#48] Arguments: [i_item_sk#33 ASC NULLS FIRST, w_warehouse_sk#34 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 13] @@ -261,12 +261,12 @@ Join type: Inner Join condition: None (46) Exchange -Input [10]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, mean#24, cov#25, w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#9 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [10]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, mean#24, cov#25, w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#47, cov#48] +Arguments: rangepartitioning(w_warehouse_sk#9 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=7] (47) Sort [codegen id : 14] -Input [10]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, mean#24, cov#25, w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#46, cov#47] -Arguments: [w_warehouse_sk#9 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, mean#24, cov#25, w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#47, cov#48] +Arguments: [w_warehouse_sk#9 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -279,22 +279,22 @@ BroadcastExchange (52) (48) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#48, d_moy#7] +Output [3]: [d_date_sk#6, d_year#49, d_moy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#48, d_moy#7] +Input [3]: [d_date_sk#6, d_year#49, d_moy#7] (50) Filter [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#48, d_moy#7] -Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#7)) AND (d_year#48 = 2001)) AND (d_moy#7 = 1)) AND isnotnull(d_date_sk#6)) +Input [3]: [d_date_sk#6, d_year#49, d_moy#7] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#7)) AND (d_year#49 = 2001)) AND (d_moy#7 = 1)) AND isnotnull(d_date_sk#6)) (51) Project [codegen id : 1] Output [2]: [d_date_sk#6, d_moy#7] -Input [3]: [d_date_sk#6, d_year#48, d_moy#7] +Input [3]: [d_date_sk#6, d_year#49, d_moy#7] (52) BroadcastExchange Input [2]: [d_date_sk#6, d_moy#7] @@ -309,22 +309,22 @@ BroadcastExchange (57) (53) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#31, d_year#49, d_moy#32] +Output [3]: [d_date_sk#31, d_year#50, d_moy#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#31, d_year#49, d_moy#32] +Input [3]: [d_date_sk#31, d_year#50, d_moy#32] (55) Filter [codegen id : 1] -Input [3]: [d_date_sk#31, d_year#49, d_moy#32] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#32)) AND (d_year#49 = 2001)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#31)) +Input [3]: [d_date_sk#31, d_year#50, d_moy#32] +Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#32)) AND (d_year#50 = 2001)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#31)) (56) Project [codegen id : 1] Output [2]: [d_date_sk#31, d_moy#32] -Input [3]: [d_date_sk#31, d_year#49, d_moy#32] +Input [3]: [d_date_sk#31, d_year#50, d_moy#32] (57) BroadcastExchange Input [2]: [d_date_sk#31, d_moy#32] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index a09240ded2c9..220598440e09 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -225,18 +225,18 @@ Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41 Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] +Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#46, avg(inv_quantity_on_hand#28)#22 AS mean#47] (39) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END +Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] +Condition : CASE WHEN (mean#47 = 0.0) THEN false ELSE ((stdev#46 / mean#47) > 1.0) END (40) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] +Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, CASE WHEN (mean#47 = 0.0) THEN null ELSE (stdev#46 / mean#47) END AS cov#48] +Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] (41) BroadcastExchange -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] +Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] (42) BroadcastHashJoin [codegen id : 10] @@ -246,12 +246,12 @@ Join type: Inner Join condition: None (43) Exchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] (44) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] +Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -264,22 +264,22 @@ BroadcastExchange (49) (45) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#48, d_moy#10] +Output [3]: [d_date_sk#9, d_year#49, d_moy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct (46) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] +Input [3]: [d_date_sk#9, d_year#49, d_moy#10] (47) Filter [codegen id : 1] -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) +Input [3]: [d_date_sk#9, d_year#49, d_moy#10] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#10)) AND (d_year#49 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) (48) Project [codegen id : 1] Output [2]: [d_date_sk#9, d_moy#10] -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] +Input [3]: [d_date_sk#9, d_year#49, d_moy#10] (49) BroadcastExchange Input [2]: [d_date_sk#9, d_moy#10] @@ -294,22 +294,22 @@ BroadcastExchange (54) (50) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#49, d_moy#35] +Output [3]: [d_date_sk#34, d_year#50, d_moy#35] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] +Input [3]: [d_date_sk#34, d_year#50, d_moy#35] (52) Filter [codegen id : 1] -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) +Input [3]: [d_date_sk#34, d_year#50, d_moy#35] +Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#35)) AND (d_year#50 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) (53) Project [codegen id : 1] Output [2]: [d_date_sk#34, d_moy#35] -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] +Input [3]: [d_date_sk#34, d_year#50, d_moy#35] (54) BroadcastExchange Input [2]: [d_date_sk#34, d_moy#35] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt index 26cda55a06fd..862f023da112 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt @@ -236,22 +236,22 @@ Input [9]: [w_warehouse_name#35, w_warehouse_sk#34, i_item_sk#33, d_moy#32, n#41 Keys [4]: [w_warehouse_name#35, w_warehouse_sk#34, i_item_sk#33, d_moy#32] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] +Results [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#46, avg(inv_quantity_on_hand#28)#22 AS mean#47] (41) Filter [codegen id : 11] -Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stdev#23, mean#24] -Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END +Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stdev#46, mean#47] +Condition : CASE WHEN (mean#47 = 0.0) THEN false ELSE ((stdev#46 / mean#47) > 1.0) END (42) Project [codegen id : 11] -Output [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] -Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stdev#23, mean#24] +Output [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#47, CASE WHEN (mean#47 = 0.0) THEN null ELSE (stdev#46 / mean#47) END AS cov#48] +Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, stdev#46, mean#47] (43) Exchange -Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#46, cov#47] +Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#47, cov#48] Arguments: hashpartitioning(i_item_sk#33, w_warehouse_sk#34, 5), ENSURE_REQUIREMENTS, [plan_id=6] (44) Sort [codegen id : 12] -Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#46, cov#47] +Input [5]: [w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#47, cov#48] Arguments: [i_item_sk#33 ASC NULLS FIRST, w_warehouse_sk#34 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 13] @@ -261,12 +261,12 @@ Join type: Inner Join condition: None (46) Exchange -Input [10]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, mean#24, cov#25, w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#9 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [10]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, mean#24, cov#25, w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#47, cov#48] +Arguments: rangepartitioning(w_warehouse_sk#9 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=7] (47) Sort [codegen id : 14] -Input [10]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, mean#24, cov#25, w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#46, cov#47] -Arguments: [w_warehouse_sk#9 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#9, i_item_sk#8, d_moy#7, mean#24, cov#25, w_warehouse_sk#34, i_item_sk#33, d_moy#32, mean#47, cov#48] +Arguments: [w_warehouse_sk#9 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -279,22 +279,22 @@ BroadcastExchange (52) (48) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#48, d_moy#7] +Output [3]: [d_date_sk#6, d_year#49, d_moy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#48, d_moy#7] +Input [3]: [d_date_sk#6, d_year#49, d_moy#7] (50) Filter [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#48, d_moy#7] -Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#7)) AND (d_year#48 = 2001)) AND (d_moy#7 = 1)) AND isnotnull(d_date_sk#6)) +Input [3]: [d_date_sk#6, d_year#49, d_moy#7] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#7)) AND (d_year#49 = 2001)) AND (d_moy#7 = 1)) AND isnotnull(d_date_sk#6)) (51) Project [codegen id : 1] Output [2]: [d_date_sk#6, d_moy#7] -Input [3]: [d_date_sk#6, d_year#48, d_moy#7] +Input [3]: [d_date_sk#6, d_year#49, d_moy#7] (52) BroadcastExchange Input [2]: [d_date_sk#6, d_moy#7] @@ -309,22 +309,22 @@ BroadcastExchange (57) (53) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#31, d_year#49, d_moy#32] +Output [3]: [d_date_sk#31, d_year#50, d_moy#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#31, d_year#49, d_moy#32] +Input [3]: [d_date_sk#31, d_year#50, d_moy#32] (55) Filter [codegen id : 1] -Input [3]: [d_date_sk#31, d_year#49, d_moy#32] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#32)) AND (d_year#49 = 2001)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#31)) +Input [3]: [d_date_sk#31, d_year#50, d_moy#32] +Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#32)) AND (d_year#50 = 2001)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#31)) (56) Project [codegen id : 1] Output [2]: [d_date_sk#31, d_moy#32] -Input [3]: [d_date_sk#31, d_year#49, d_moy#32] +Input [3]: [d_date_sk#31, d_year#50, d_moy#32] (57) BroadcastExchange Input [2]: [d_date_sk#31, d_moy#32] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index e7332b566d3b..585e74886055 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -225,18 +225,18 @@ Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41 Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] +Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#46, avg(inv_quantity_on_hand#28)#22 AS mean#47] (39) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END +Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] +Condition : CASE WHEN (mean#47 = 0.0) THEN false ELSE ((stdev#46 / mean#47) > 1.0) END (40) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] +Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, CASE WHEN (mean#47 = 0.0) THEN null ELSE (stdev#46 / mean#47) END AS cov#48] +Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] (41) BroadcastExchange -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] +Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] (42) BroadcastHashJoin [codegen id : 10] @@ -246,12 +246,12 @@ Join type: Inner Join condition: None (43) Exchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] (44) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] +Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -264,22 +264,22 @@ BroadcastExchange (49) (45) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#48, d_moy#10] +Output [3]: [d_date_sk#9, d_year#49, d_moy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct (46) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] +Input [3]: [d_date_sk#9, d_year#49, d_moy#10] (47) Filter [codegen id : 1] -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) +Input [3]: [d_date_sk#9, d_year#49, d_moy#10] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#10)) AND (d_year#49 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) (48) Project [codegen id : 1] Output [2]: [d_date_sk#9, d_moy#10] -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] +Input [3]: [d_date_sk#9, d_year#49, d_moy#10] (49) BroadcastExchange Input [2]: [d_date_sk#9, d_moy#10] @@ -294,22 +294,22 @@ BroadcastExchange (54) (50) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#49, d_moy#35] +Output [3]: [d_date_sk#34, d_year#50, d_moy#35] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] +Input [3]: [d_date_sk#34, d_year#50, d_moy#35] (52) Filter [codegen id : 1] -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) +Input [3]: [d_date_sk#34, d_year#50, d_moy#35] +Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#35)) AND (d_year#50 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) (53) Project [codegen id : 1] Output [2]: [d_date_sk#34, d_moy#35] -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] +Input [3]: [d_date_sk#34, d_year#50, d_moy#35] (54) BroadcastExchange Input [2]: [d_date_sk#34, d_moy#35] 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 06a84d74948c..d8eacfca11bc 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 @@ -211,74 +211,74 @@ Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_yea Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] -Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#18] +Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#30] (36) Exchange -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=7] (37) Sort [codegen id : 21] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 (38) Window -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#18 AS sum_sales#31, rn#30] -Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18, rn#30] +Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] (40) Exchange -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1), 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1), 5), ENSURE_REQUIREMENTS, [plan_id=8] (41) Sort [codegen id : 23] -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] -Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, (rn#30 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, (rn#31 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [5]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, rn#20] -Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1)] +Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] Join type: Inner Join condition: None (43) Project [codegen id : 24] -Output [10]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] +Output [10]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30] +Input [15]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] (44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] (45) Sort [codegen id : 33] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 (46) Window -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#18 AS sum_sales#39, rn#38] -Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18, rn#38] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] (48) Exchange -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] -Arguments: hashpartitioning(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1), 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Arguments: hashpartitioning(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1), 5), ENSURE_REQUIREMENTS, [plan_id=9] (49) Sort [codegen id : 35] -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] -Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, (rn#38 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, (rn#39 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [5]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, rn#20] -Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1)] +Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] Join type: Inner Join condition: None (51) Project [codegen id : 36] -Output [10]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] +Output [10]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 AS nsum#41] +Input [16]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] (52) TakeOrderedAndProject Input [10]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] 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 e5b3c07655fd..0854f607101f 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 @@ -184,66 +184,66 @@ Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_yea Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] -Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#18] +Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#30] (31) Exchange -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] (32) Sort [codegen id : 13] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 (33) Window -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#18 AS sum_sales#31, rn#30] -Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18, rn#30] +Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] (35) BroadcastExchange -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] 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), [plan_id=6] (36) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] -Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1)] +Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] Join type: Inner Join condition: None (37) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] (38) ReusedExchange [Reuses operator id: 31] -Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] (39) Sort [codegen id : 20] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 (40) Window -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#18 AS sum_sales#39, rn#38] -Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18, rn#38] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] (42) BroadcastExchange -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] 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), [plan_id=7] (43) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] -Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1)] +Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] Join type: Inner Join condition: None (44) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 AS nsum#41] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] (45) TakeOrderedAndProject Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] 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 70ca72c14393..f027be356ea8 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 @@ -211,74 +211,74 @@ Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] -Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#17] +Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#28] (36) Exchange -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=7] (37) Sort [codegen id : 21] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 (38) Window -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#17 AS sum_sales#29, rn#28] -Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17, rn#28] +Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] (40) Exchange -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, (rn#28 + 1), 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, (rn#29 + 1), 5), ENSURE_REQUIREMENTS, [plan_id=8] (41) Sort [codegen id : 23] -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] -Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, (rn#28 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, (rn#29 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [4]: [i_category#13, i_brand#12, cc_name#10, rn#19] -Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#28 + 1)] +Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#29 + 1)] Join type: Inner Join condition: None (43) Project [codegen id : 24] -Output [9]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] +Output [9]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] +Input [13]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] (44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] (45) Sort [codegen id : 33] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 (46) Window -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#17 AS sum_sales#36, rn#35] -Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] (48) Exchange -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] -Arguments: hashpartitioning(i_category#30, i_brand#31, cc_name#32, (rn#35 - 1), 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Arguments: hashpartitioning(i_category#30, i_brand#31, cc_name#32, (rn#36 - 1), 5), ENSURE_REQUIREMENTS, [plan_id=9] (49) Sort [codegen id : 35] -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, (rn#35 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, (rn#36 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [4]: [i_category#13, i_brand#12, cc_name#10, rn#19] -Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#35 - 1)] +Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#36 - 1)] Join type: Inner Join condition: None (51) Project [codegen id : 36] -Output [9]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] +Output [9]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 AS nsum#38] +Input [14]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] (52) TakeOrderedAndProject Input [9]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] 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 d7acef9829ce..9681432f0674 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 @@ -184,66 +184,66 @@ Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] -Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#17] +Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#28] (31) Exchange -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] (32) Sort [codegen id : 13] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 (33) Window -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#17 AS sum_sales#29, rn#28] -Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17, rn#28] +Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] (35) BroadcastExchange -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] (36) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] -Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#28 + 1)] +Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#29 + 1)] Join type: Inner Join condition: None (37) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] (38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] (39) Sort [codegen id : 20] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 (40) Window -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#17 AS sum_sales#36, rn#35] -Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] (42) BroadcastExchange -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] (43) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] -Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#35 - 1)] +Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#36 - 1)] Join type: Inner Join condition: None (44) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 AS nsum#38] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] (45) TakeOrderedAndProject Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt index 25704ce2a276..d8de4e4c4f8c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt @@ -517,204 +517,204 @@ Join type: LeftOuter Join condition: None (86) Project [codegen id : 32] -Output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#20, (cs_ext_sales_price#71 - coalesce(cr_return_amount#84, 0.00)) AS sales_amt#21] +Output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#85, (cs_ext_sales_price#71 - coalesce(cr_return_amount#84, 0.00)) AS sales_amt#86] Input [13]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80, cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] (87) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#89), dynamicpruningexpression(ss_sold_date_sk#89 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (88) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] (89) Filter [codegen id : 35] -Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Condition : isnotnull(ss_item_sk#85) +Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +Condition : isnotnull(ss_item_sk#87) (90) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94] +Output [5]: [i_item_sk#92, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96] (91) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#85] -Right keys [1]: [i_item_sk#90] +Left keys [1]: [ss_item_sk#87] +Right keys [1]: [i_item_sk#92] Join type: Inner Join condition: None (92) Project [codegen id : 35] -Output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94] -Input [10]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94] +Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96] +Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#92, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96] (93) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#95, d_year#96] +Output [2]: [d_date_sk#97, d_year#98] (94) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#89] -Right keys [1]: [d_date_sk#95] +Left keys [1]: [ss_sold_date_sk#91] +Right keys [1]: [d_date_sk#97] Join type: Inner Join condition: None (95) Project [codegen id : 35] -Output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96] -Input [11]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_date_sk#95, d_year#96] +Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98] +Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_date_sk#97, d_year#98] (96) Exchange -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96] -Arguments: hashpartitioning(ss_ticket_number#86, ss_item_sk#85, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98] +Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=12] (97) Sort [codegen id : 36] -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96] -Arguments: [ss_ticket_number#86 ASC NULLS FIRST, ss_item_sk#85 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98] +Arguments: [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST], false, 0 (98) ReusedExchange [Reuses operator id: 39] -Output [4]: [sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100] +Output [4]: [sr_item_sk#99, sr_ticket_number#100, sr_return_quantity#101, sr_return_amt#102] (99) Sort [codegen id : 38] -Input [4]: [sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100] -Arguments: [sr_ticket_number#98 ASC NULLS FIRST, sr_item_sk#97 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#99, sr_ticket_number#100, sr_return_quantity#101, sr_return_amt#102] +Arguments: [sr_ticket_number#100 ASC NULLS FIRST, sr_item_sk#99 ASC NULLS FIRST], false, 0 (100) SortMergeJoin [codegen id : 39] -Left keys [2]: [ss_ticket_number#86, ss_item_sk#85] -Right keys [2]: [sr_ticket_number#98, sr_item_sk#97] +Left keys [2]: [ss_ticket_number#88, ss_item_sk#87] +Right keys [2]: [sr_ticket_number#100, sr_item_sk#99] Join type: LeftOuter Join condition: None (101) Project [codegen id : 39] -Output [7]: [d_year#96, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, (ss_quantity#87 - coalesce(sr_return_quantity#99, 0)) AS sales_cnt#39, (ss_ext_sales_price#88 - coalesce(sr_return_amt#100, 0.00)) AS sales_amt#40] -Input [13]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96, sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100] +Output [7]: [d_year#98, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, (ss_quantity#89 - coalesce(sr_return_quantity#101, 0)) AS sales_cnt#103, (ss_ext_sales_price#90 - coalesce(sr_return_amt#102, 0.00)) AS sales_amt#104] +Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98, sr_item_sk#99, sr_ticket_number#100, sr_return_quantity#101, sr_return_amt#102] (102) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] +Output [5]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#105), dynamicpruningexpression(ws_sold_date_sk#105 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ws_sold_date_sk#109), dynamicpruningexpression(ws_sold_date_sk#109 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (103) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] +Input [5]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109] (104) Filter [codegen id : 42] -Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] -Condition : isnotnull(ws_item_sk#101) +Input [5]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109] +Condition : isnotnull(ws_item_sk#105) (105) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] (106) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#101] -Right keys [1]: [i_item_sk#106] +Left keys [1]: [ws_item_sk#105] +Right keys [1]: [i_item_sk#110] Join type: Inner Join condition: None (107) Project [codegen id : 42] -Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Input [10]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Output [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Input [10]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] (108) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#111, d_year#112] +Output [2]: [d_date_sk#115, d_year#116] (109) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#105] -Right keys [1]: [d_date_sk#111] +Left keys [1]: [ws_sold_date_sk#109] +Right keys [1]: [d_date_sk#115] Join type: Inner Join condition: None (110) Project [codegen id : 42] -Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Input [11]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] +Output [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Input [11]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] (111) Exchange -Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#102, ws_item_sk#101, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Input [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: hashpartitioning(ws_order_number#106, ws_item_sk#105, 5), ENSURE_REQUIREMENTS, [plan_id=13] (112) Sort [codegen id : 43] -Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_order_number#102 ASC NULLS FIRST, ws_item_sk#101 ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: [ws_order_number#106 ASC NULLS FIRST, ws_item_sk#105 ASC NULLS FIRST], false, 0 (113) ReusedExchange [Reuses operator id: 58] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] (114) Sort [codegen id : 45] -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST], false, 0 (115) SortMergeJoin [codegen id : 46] -Left keys [2]: [ws_order_number#102, ws_item_sk#101] -Right keys [2]: [wr_order_number#114, wr_item_sk#113] +Left keys [2]: [ws_order_number#106, ws_item_sk#105] +Right keys [2]: [wr_order_number#118, wr_item_sk#117] Join type: LeftOuter Join condition: None (116) Project [codegen id : 46] -Output [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#103 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#58, (ws_ext_sales_price#104 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#59] -Input [13]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Output [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#107 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#121, (ws_ext_sales_price#108 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#122] +Input [13]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] (117) Union (118) HashAggregate [codegen id : 47] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] (119) Exchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86, 5), ENSURE_REQUIREMENTS, [plan_id=14] (120) HashAggregate [codegen id : 48] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] (121) HashAggregate [codegen id : 48] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum#60, sum#117] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118] +Functions [2]: [partial_sum(sales_cnt#85), partial_sum(UnscaledValue(sales_amt#86))] +Aggregate Attributes [2]: [sum#123, sum#124] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] (122) Exchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, [plan_id=15] (123) HashAggregate [codegen id : 49] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#64, sum(UnscaledValue(sales_amt#21))#65] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum(sales_cnt#20)#64 AS sales_cnt#119, MakeDecimal(sum(UnscaledValue(sales_amt#21))#65,18,2) AS sales_amt#120] +Functions [2]: [sum(sales_cnt#85), sum(UnscaledValue(sales_amt#86))] +Aggregate Attributes [2]: [sum(sales_cnt#85)#64, sum(UnscaledValue(sales_amt#86))#65] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum(sales_cnt#85)#64 AS sales_cnt#127, MakeDecimal(sum(UnscaledValue(sales_amt#86))#65,18,2) AS sales_amt#128] (124) Filter [codegen id : 49] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] -Condition : isnotnull(sales_cnt#119) +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] +Condition : isnotnull(sales_cnt#127) (125) Exchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, [plan_id=16] (126) Sort [codegen id : 50] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] Arguments: [i_brand_id#75 ASC NULLS FIRST, i_class_id#76 ASC NULLS FIRST, i_category_id#77 ASC NULLS FIRST, i_manufact_id#78 ASC NULLS FIRST], false, 0 (127) SortMergeJoin [codegen id : 51] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Right keys [4]: [i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] Join type: Inner -Join condition: ((cast(sales_cnt#66 as decimal(17,2)) / cast(sales_cnt#119 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#66 as decimal(17,2)) / cast(sales_cnt#127 as decimal(17,2))) < 0.90000000000000000000) (128) Project [codegen id : 51] -Output [10]: [d_year#80 AS prev_year#121, d_year#14 AS year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#119 AS prev_yr_cnt#123, sales_cnt#66 AS curr_yr_cnt#124, (sales_cnt#66 - sales_cnt#119) AS sales_cnt_diff#125, (sales_amt#67 - sales_amt#120) AS sales_amt_diff#126] -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67, d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] +Output [10]: [d_year#80 AS prev_year#129, d_year#14 AS year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#127 AS prev_yr_cnt#131, sales_cnt#66 AS curr_yr_cnt#132, (sales_cnt#66 - sales_cnt#127) AS sales_cnt_diff#133, (sales_amt#67 - sales_amt#128) AS sales_amt_diff#134] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67, d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] (129) TakeOrderedAndProject -Input [10]: [prev_year#121, year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#123, curr_yr_cnt#124, sales_cnt_diff#125, sales_amt_diff#126] -Arguments: 100, [sales_cnt_diff#125 ASC NULLS FIRST], [prev_year#121, year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#123, curr_yr_cnt#124, sales_cnt_diff#125, sales_amt_diff#126] +Input [10]: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] +Arguments: 100, [sales_cnt_diff#133 ASC NULLS FIRST], [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] ===== Subqueries ===== @@ -772,8 +772,8 @@ Condition : ((isnotnull(d_year#80) AND (d_year#80 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#79, d_year#80] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#73 +Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#73 -Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#105 IN dynamicpruning#73 +Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#109 IN dynamicpruning#73 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index 25704ce2a276..d8de4e4c4f8c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -517,204 +517,204 @@ Join type: LeftOuter Join condition: None (86) Project [codegen id : 32] -Output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#20, (cs_ext_sales_price#71 - coalesce(cr_return_amount#84, 0.00)) AS sales_amt#21] +Output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#85, (cs_ext_sales_price#71 - coalesce(cr_return_amount#84, 0.00)) AS sales_amt#86] Input [13]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80, cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] (87) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#89), dynamicpruningexpression(ss_sold_date_sk#89 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (88) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] (89) Filter [codegen id : 35] -Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Condition : isnotnull(ss_item_sk#85) +Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +Condition : isnotnull(ss_item_sk#87) (90) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94] +Output [5]: [i_item_sk#92, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96] (91) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#85] -Right keys [1]: [i_item_sk#90] +Left keys [1]: [ss_item_sk#87] +Right keys [1]: [i_item_sk#92] Join type: Inner Join condition: None (92) Project [codegen id : 35] -Output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94] -Input [10]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94] +Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96] +Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#92, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96] (93) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#95, d_year#96] +Output [2]: [d_date_sk#97, d_year#98] (94) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#89] -Right keys [1]: [d_date_sk#95] +Left keys [1]: [ss_sold_date_sk#91] +Right keys [1]: [d_date_sk#97] Join type: Inner Join condition: None (95) Project [codegen id : 35] -Output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96] -Input [11]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_date_sk#95, d_year#96] +Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98] +Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_date_sk#97, d_year#98] (96) Exchange -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96] -Arguments: hashpartitioning(ss_ticket_number#86, ss_item_sk#85, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98] +Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=12] (97) Sort [codegen id : 36] -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96] -Arguments: [ss_ticket_number#86 ASC NULLS FIRST, ss_item_sk#85 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98] +Arguments: [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST], false, 0 (98) ReusedExchange [Reuses operator id: 39] -Output [4]: [sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100] +Output [4]: [sr_item_sk#99, sr_ticket_number#100, sr_return_quantity#101, sr_return_amt#102] (99) Sort [codegen id : 38] -Input [4]: [sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100] -Arguments: [sr_ticket_number#98 ASC NULLS FIRST, sr_item_sk#97 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#99, sr_ticket_number#100, sr_return_quantity#101, sr_return_amt#102] +Arguments: [sr_ticket_number#100 ASC NULLS FIRST, sr_item_sk#99 ASC NULLS FIRST], false, 0 (100) SortMergeJoin [codegen id : 39] -Left keys [2]: [ss_ticket_number#86, ss_item_sk#85] -Right keys [2]: [sr_ticket_number#98, sr_item_sk#97] +Left keys [2]: [ss_ticket_number#88, ss_item_sk#87] +Right keys [2]: [sr_ticket_number#100, sr_item_sk#99] Join type: LeftOuter Join condition: None (101) Project [codegen id : 39] -Output [7]: [d_year#96, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, (ss_quantity#87 - coalesce(sr_return_quantity#99, 0)) AS sales_cnt#39, (ss_ext_sales_price#88 - coalesce(sr_return_amt#100, 0.00)) AS sales_amt#40] -Input [13]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96, sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100] +Output [7]: [d_year#98, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, (ss_quantity#89 - coalesce(sr_return_quantity#101, 0)) AS sales_cnt#103, (ss_ext_sales_price#90 - coalesce(sr_return_amt#102, 0.00)) AS sales_amt#104] +Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98, sr_item_sk#99, sr_ticket_number#100, sr_return_quantity#101, sr_return_amt#102] (102) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] +Output [5]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#105), dynamicpruningexpression(ws_sold_date_sk#105 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ws_sold_date_sk#109), dynamicpruningexpression(ws_sold_date_sk#109 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (103) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] +Input [5]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109] (104) Filter [codegen id : 42] -Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] -Condition : isnotnull(ws_item_sk#101) +Input [5]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109] +Condition : isnotnull(ws_item_sk#105) (105) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] (106) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#101] -Right keys [1]: [i_item_sk#106] +Left keys [1]: [ws_item_sk#105] +Right keys [1]: [i_item_sk#110] Join type: Inner Join condition: None (107) Project [codegen id : 42] -Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Input [10]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Output [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Input [10]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] (108) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#111, d_year#112] +Output [2]: [d_date_sk#115, d_year#116] (109) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#105] -Right keys [1]: [d_date_sk#111] +Left keys [1]: [ws_sold_date_sk#109] +Right keys [1]: [d_date_sk#115] Join type: Inner Join condition: None (110) Project [codegen id : 42] -Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Input [11]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] +Output [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Input [11]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] (111) Exchange -Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#102, ws_item_sk#101, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Input [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: hashpartitioning(ws_order_number#106, ws_item_sk#105, 5), ENSURE_REQUIREMENTS, [plan_id=13] (112) Sort [codegen id : 43] -Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_order_number#102 ASC NULLS FIRST, ws_item_sk#101 ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: [ws_order_number#106 ASC NULLS FIRST, ws_item_sk#105 ASC NULLS FIRST], false, 0 (113) ReusedExchange [Reuses operator id: 58] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] (114) Sort [codegen id : 45] -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST], false, 0 (115) SortMergeJoin [codegen id : 46] -Left keys [2]: [ws_order_number#102, ws_item_sk#101] -Right keys [2]: [wr_order_number#114, wr_item_sk#113] +Left keys [2]: [ws_order_number#106, ws_item_sk#105] +Right keys [2]: [wr_order_number#118, wr_item_sk#117] Join type: LeftOuter Join condition: None (116) Project [codegen id : 46] -Output [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#103 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#58, (ws_ext_sales_price#104 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#59] -Input [13]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Output [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#107 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#121, (ws_ext_sales_price#108 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#122] +Input [13]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] (117) Union (118) HashAggregate [codegen id : 47] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] (119) Exchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86, 5), ENSURE_REQUIREMENTS, [plan_id=14] (120) HashAggregate [codegen id : 48] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] (121) HashAggregate [codegen id : 48] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum#60, sum#117] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118] +Functions [2]: [partial_sum(sales_cnt#85), partial_sum(UnscaledValue(sales_amt#86))] +Aggregate Attributes [2]: [sum#123, sum#124] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] (122) Exchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, [plan_id=15] (123) HashAggregate [codegen id : 49] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#64, sum(UnscaledValue(sales_amt#21))#65] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum(sales_cnt#20)#64 AS sales_cnt#119, MakeDecimal(sum(UnscaledValue(sales_amt#21))#65,18,2) AS sales_amt#120] +Functions [2]: [sum(sales_cnt#85), sum(UnscaledValue(sales_amt#86))] +Aggregate Attributes [2]: [sum(sales_cnt#85)#64, sum(UnscaledValue(sales_amt#86))#65] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum(sales_cnt#85)#64 AS sales_cnt#127, MakeDecimal(sum(UnscaledValue(sales_amt#86))#65,18,2) AS sales_amt#128] (124) Filter [codegen id : 49] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] -Condition : isnotnull(sales_cnt#119) +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] +Condition : isnotnull(sales_cnt#127) (125) Exchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, [plan_id=16] (126) Sort [codegen id : 50] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] Arguments: [i_brand_id#75 ASC NULLS FIRST, i_class_id#76 ASC NULLS FIRST, i_category_id#77 ASC NULLS FIRST, i_manufact_id#78 ASC NULLS FIRST], false, 0 (127) SortMergeJoin [codegen id : 51] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Right keys [4]: [i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] Join type: Inner -Join condition: ((cast(sales_cnt#66 as decimal(17,2)) / cast(sales_cnt#119 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#66 as decimal(17,2)) / cast(sales_cnt#127 as decimal(17,2))) < 0.90000000000000000000) (128) Project [codegen id : 51] -Output [10]: [d_year#80 AS prev_year#121, d_year#14 AS year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#119 AS prev_yr_cnt#123, sales_cnt#66 AS curr_yr_cnt#124, (sales_cnt#66 - sales_cnt#119) AS sales_cnt_diff#125, (sales_amt#67 - sales_amt#120) AS sales_amt_diff#126] -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67, d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] +Output [10]: [d_year#80 AS prev_year#129, d_year#14 AS year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#127 AS prev_yr_cnt#131, sales_cnt#66 AS curr_yr_cnt#132, (sales_cnt#66 - sales_cnt#127) AS sales_cnt_diff#133, (sales_amt#67 - sales_amt#128) AS sales_amt_diff#134] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67, d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] (129) TakeOrderedAndProject -Input [10]: [prev_year#121, year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#123, curr_yr_cnt#124, sales_cnt_diff#125, sales_amt_diff#126] -Arguments: 100, [sales_cnt_diff#125 ASC NULLS FIRST], [prev_year#121, year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#123, curr_yr_cnt#124, sales_cnt_diff#125, sales_amt_diff#126] +Input [10]: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] +Arguments: 100, [sales_cnt_diff#133 ASC NULLS FIRST], [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] ===== Subqueries ===== @@ -772,8 +772,8 @@ Condition : ((isnotnull(d_year#80) AND (d_year#80 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#79, d_year#80] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#73 +Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#73 -Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#105 IN dynamicpruning#73 +Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#109 IN dynamicpruning#73 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt index 96bed479d2e0..4bf7de791b27 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt @@ -175,125 +175,125 @@ Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#21, cou Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(qoh#18)] Aggregate Attributes [1]: [avg(qoh#18)#23] -Results [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, avg(qoh#18)#23 AS qoh#24] +Results [5]: [i_product_name#12 AS i_product_name#24, i_brand#9 AS i_brand#25, i_class#10 AS i_class#26, i_category#11 AS i_category#27, avg(qoh#18)#23 AS qoh#28] (27) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30] +Output [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] (28) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30] -Keys [4]: [i_product_name#25, i_brand#26, i_class#27, i_category#28] -Functions [1]: [avg(inv_quantity_on_hand#31)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#31)#17] -Results [4]: [i_product_name#25, i_brand#26, i_class#27, avg(inv_quantity_on_hand#31)#17 AS qoh#32] +Input [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] +Keys [4]: [i_product_name#29, i_brand#30, i_class#31, i_category#32] +Functions [1]: [avg(inv_quantity_on_hand#35)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#17] +Results [4]: [i_product_name#29, i_brand#30, i_class#31, avg(inv_quantity_on_hand#35)#17 AS qoh#36] (29) HashAggregate [codegen id : 16] -Input [4]: [i_product_name#25, i_brand#26, i_class#27, qoh#32] -Keys [3]: [i_product_name#25, i_brand#26, i_class#27] -Functions [1]: [partial_avg(qoh#32)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] +Input [4]: [i_product_name#29, i_brand#30, i_class#31, qoh#36] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [partial_avg(qoh#36)] +Aggregate Attributes [2]: [sum#37, count#38] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] (30) Exchange -Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] -Arguments: hashpartitioning(i_product_name#25, i_brand#26, i_class#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Arguments: hashpartitioning(i_product_name#29, i_brand#30, i_class#31, 5), ENSURE_REQUIREMENTS, [plan_id=5] (31) HashAggregate [codegen id : 17] -Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] -Keys [3]: [i_product_name#25, i_brand#26, i_class#27] -Functions [1]: [avg(qoh#32)] -Aggregate Attributes [1]: [avg(qoh#32)#37] -Results [5]: [i_product_name#25, i_brand#26, i_class#27, null AS i_category#38, avg(qoh#32)#37 AS qoh#39] +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [avg(qoh#36)] +Aggregate Attributes [1]: [avg(qoh#36)#41] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, null AS i_category#42, avg(qoh#36)#41 AS qoh#43] (32) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45] +Output [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] (33) HashAggregate [codegen id : 25] -Input [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45] -Keys [4]: [i_product_name#40, i_brand#41, i_class#42, i_category#43] -Functions [1]: [avg(inv_quantity_on_hand#46)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#17] -Results [3]: [i_product_name#40, i_brand#41, avg(inv_quantity_on_hand#46)#17 AS qoh#47] +Input [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] +Keys [4]: [i_product_name#44, i_brand#45, i_class#46, i_category#47] +Functions [1]: [avg(inv_quantity_on_hand#50)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#50)#17] +Results [3]: [i_product_name#44, i_brand#45, avg(inv_quantity_on_hand#50)#17 AS qoh#51] (34) HashAggregate [codegen id : 25] -Input [3]: [i_product_name#40, i_brand#41, qoh#47] -Keys [2]: [i_product_name#40, i_brand#41] -Functions [1]: [partial_avg(qoh#47)] -Aggregate Attributes [2]: [sum#48, count#49] -Results [4]: [i_product_name#40, i_brand#41, sum#50, count#51] +Input [3]: [i_product_name#44, i_brand#45, qoh#51] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [partial_avg(qoh#51)] +Aggregate Attributes [2]: [sum#52, count#53] +Results [4]: [i_product_name#44, i_brand#45, sum#54, count#55] (35) Exchange -Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51] -Arguments: hashpartitioning(i_product_name#40, i_brand#41, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Arguments: hashpartitioning(i_product_name#44, i_brand#45, 5), ENSURE_REQUIREMENTS, [plan_id=6] (36) HashAggregate [codegen id : 26] -Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51] -Keys [2]: [i_product_name#40, i_brand#41] -Functions [1]: [avg(qoh#47)] -Aggregate Attributes [1]: [avg(qoh#47)#52] -Results [5]: [i_product_name#40, i_brand#41, null AS i_class#53, null AS i_category#54, avg(qoh#47)#52 AS qoh#55] +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [avg(qoh#51)] +Aggregate Attributes [1]: [avg(qoh#51)#56] +Results [5]: [i_product_name#44, i_brand#45, null AS i_class#57, null AS i_category#58, avg(qoh#51)#56 AS qoh#59] (37) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61] +Output [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] (38) HashAggregate [codegen id : 34] -Input [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61] -Keys [4]: [i_product_name#56, i_brand#57, i_class#58, i_category#59] -Functions [1]: [avg(inv_quantity_on_hand#62)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#62)#17] -Results [2]: [i_product_name#56, avg(inv_quantity_on_hand#62)#17 AS qoh#63] +Input [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] +Keys [4]: [i_product_name#60, i_brand#61, i_class#62, i_category#63] +Functions [1]: [avg(inv_quantity_on_hand#66)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#66)#17] +Results [2]: [i_product_name#60, avg(inv_quantity_on_hand#66)#17 AS qoh#67] (39) HashAggregate [codegen id : 34] -Input [2]: [i_product_name#56, qoh#63] -Keys [1]: [i_product_name#56] -Functions [1]: [partial_avg(qoh#63)] -Aggregate Attributes [2]: [sum#64, count#65] -Results [3]: [i_product_name#56, sum#66, count#67] +Input [2]: [i_product_name#60, qoh#67] +Keys [1]: [i_product_name#60] +Functions [1]: [partial_avg(qoh#67)] +Aggregate Attributes [2]: [sum#68, count#69] +Results [3]: [i_product_name#60, sum#70, count#71] (40) Exchange -Input [3]: [i_product_name#56, sum#66, count#67] -Arguments: hashpartitioning(i_product_name#56, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [3]: [i_product_name#60, sum#70, count#71] +Arguments: hashpartitioning(i_product_name#60, 5), ENSURE_REQUIREMENTS, [plan_id=7] (41) HashAggregate [codegen id : 35] -Input [3]: [i_product_name#56, sum#66, count#67] -Keys [1]: [i_product_name#56] -Functions [1]: [avg(qoh#63)] -Aggregate Attributes [1]: [avg(qoh#63)#68] -Results [5]: [i_product_name#56, null AS i_brand#69, null AS i_class#70, null AS i_category#71, avg(qoh#63)#68 AS qoh#72] +Input [3]: [i_product_name#60, sum#70, count#71] +Keys [1]: [i_product_name#60] +Functions [1]: [avg(qoh#67)] +Aggregate Attributes [1]: [avg(qoh#67)#72] +Results [5]: [i_product_name#60, null AS i_brand#73, null AS i_class#74, null AS i_category#75, avg(qoh#67)#72 AS qoh#76] (42) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78] +Output [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] (43) HashAggregate [codegen id : 43] -Input [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78] -Keys [4]: [i_product_name#73, i_brand#74, i_class#75, i_category#76] -Functions [1]: [avg(inv_quantity_on_hand#79)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#79)#17] -Results [1]: [avg(inv_quantity_on_hand#79)#17 AS qoh#80] +Input [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] +Keys [4]: [i_product_name#77, i_brand#78, i_class#79, i_category#80] +Functions [1]: [avg(inv_quantity_on_hand#83)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#83)#17] +Results [1]: [avg(inv_quantity_on_hand#83)#17 AS qoh#84] (44) HashAggregate [codegen id : 43] -Input [1]: [qoh#80] +Input [1]: [qoh#84] Keys: [] -Functions [1]: [partial_avg(qoh#80)] -Aggregate Attributes [2]: [sum#81, count#82] -Results [2]: [sum#83, count#84] +Functions [1]: [partial_avg(qoh#84)] +Aggregate Attributes [2]: [sum#85, count#86] +Results [2]: [sum#87, count#88] (45) Exchange -Input [2]: [sum#83, count#84] +Input [2]: [sum#87, count#88] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] (46) HashAggregate [codegen id : 44] -Input [2]: [sum#83, count#84] +Input [2]: [sum#87, count#88] Keys: [] -Functions [1]: [avg(qoh#80)] -Aggregate Attributes [1]: [avg(qoh#80)#85] -Results [5]: [null AS i_product_name#86, null AS i_brand#87, null AS i_class#88, null AS i_category#89, avg(qoh#80)#85 AS qoh#90] +Functions [1]: [avg(qoh#84)] +Aggregate Attributes [1]: [avg(qoh#84)#89] +Results [5]: [null AS i_product_name#90, null AS i_brand#91, null AS i_class#92, null AS i_category#93, avg(qoh#84)#89 AS qoh#94] (47) Union (48) TakeOrderedAndProject -Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#24] -Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_brand#9 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#24] +Input [5]: [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] +Arguments: 100, [qoh#28 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_class#26 ASC NULLS FIRST, i_category#27 ASC NULLS FIRST], [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] ===== Subqueries ===== @@ -306,22 +306,22 @@ BroadcastExchange (53) (49) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_month_seq#91] +Output [2]: [d_date_sk#7, d_month_seq#95] 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), IsNotNull(d_date_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#7, d_month_seq#91] +Input [2]: [d_date_sk#7, d_month_seq#95] (51) Filter [codegen id : 1] -Input [2]: [d_date_sk#7, d_month_seq#91] -Condition : (((isnotnull(d_month_seq#91) AND (d_month_seq#91 >= 1212)) AND (d_month_seq#91 <= 1223)) AND isnotnull(d_date_sk#7)) +Input [2]: [d_date_sk#7, d_month_seq#95] +Condition : (((isnotnull(d_month_seq#95) AND (d_month_seq#95 >= 1212)) AND (d_month_seq#95 <= 1223)) AND isnotnull(d_date_sk#7)) (52) Project [codegen id : 1] Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_month_seq#91] +Input [2]: [d_date_sk#7, d_month_seq#95] (53) BroadcastExchange Input [1]: [d_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt index 0c4267b3ca51..042f946b8fca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union WholeStageCodegen (8) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt index 4b8993f370f4..8aab8e91acfc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt @@ -160,125 +160,125 @@ Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, coun Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(qoh#18)] Aggregate Attributes [1]: [avg(qoh#18)#23] -Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(qoh#18)#23 AS qoh#24] +Results [5]: [i_product_name#11 AS i_product_name#24, i_brand#8 AS i_brand#25, i_class#9 AS i_class#26, i_category#10 AS i_category#27, avg(qoh#18)#23 AS qoh#28] (24) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30] +Output [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] (25) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30] -Keys [4]: [i_product_name#25, i_brand#26, i_class#27, i_category#28] -Functions [1]: [avg(inv_quantity_on_hand#31)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#31)#17] -Results [4]: [i_product_name#25, i_brand#26, i_class#27, avg(inv_quantity_on_hand#31)#17 AS qoh#32] +Input [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] +Keys [4]: [i_product_name#29, i_brand#30, i_class#31, i_category#32] +Functions [1]: [avg(inv_quantity_on_hand#35)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#17] +Results [4]: [i_product_name#29, i_brand#30, i_class#31, avg(inv_quantity_on_hand#35)#17 AS qoh#36] (26) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#25, i_brand#26, i_class#27, qoh#32] -Keys [3]: [i_product_name#25, i_brand#26, i_class#27] -Functions [1]: [partial_avg(qoh#32)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] +Input [4]: [i_product_name#29, i_brand#30, i_class#31, qoh#36] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [partial_avg(qoh#36)] +Aggregate Attributes [2]: [sum#37, count#38] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] (27) Exchange -Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] -Arguments: hashpartitioning(i_product_name#25, i_brand#26, i_class#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Arguments: hashpartitioning(i_product_name#29, i_brand#30, i_class#31, 5), ENSURE_REQUIREMENTS, [plan_id=4] (28) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] -Keys [3]: [i_product_name#25, i_brand#26, i_class#27] -Functions [1]: [avg(qoh#32)] -Aggregate Attributes [1]: [avg(qoh#32)#37] -Results [5]: [i_product_name#25, i_brand#26, i_class#27, null AS i_category#38, avg(qoh#32)#37 AS qoh#39] +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [avg(qoh#36)] +Aggregate Attributes [1]: [avg(qoh#36)#41] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, null AS i_category#42, avg(qoh#36)#41 AS qoh#43] (29) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45] +Output [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] (30) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45] -Keys [4]: [i_product_name#40, i_brand#41, i_class#42, i_category#43] -Functions [1]: [avg(inv_quantity_on_hand#46)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#17] -Results [3]: [i_product_name#40, i_brand#41, avg(inv_quantity_on_hand#46)#17 AS qoh#47] +Input [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] +Keys [4]: [i_product_name#44, i_brand#45, i_class#46, i_category#47] +Functions [1]: [avg(inv_quantity_on_hand#50)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#50)#17] +Results [3]: [i_product_name#44, i_brand#45, avg(inv_quantity_on_hand#50)#17 AS qoh#51] (31) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#40, i_brand#41, qoh#47] -Keys [2]: [i_product_name#40, i_brand#41] -Functions [1]: [partial_avg(qoh#47)] -Aggregate Attributes [2]: [sum#48, count#49] -Results [4]: [i_product_name#40, i_brand#41, sum#50, count#51] +Input [3]: [i_product_name#44, i_brand#45, qoh#51] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [partial_avg(qoh#51)] +Aggregate Attributes [2]: [sum#52, count#53] +Results [4]: [i_product_name#44, i_brand#45, sum#54, count#55] (32) Exchange -Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51] -Arguments: hashpartitioning(i_product_name#40, i_brand#41, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Arguments: hashpartitioning(i_product_name#44, i_brand#45, 5), ENSURE_REQUIREMENTS, [plan_id=5] (33) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51] -Keys [2]: [i_product_name#40, i_brand#41] -Functions [1]: [avg(qoh#47)] -Aggregate Attributes [1]: [avg(qoh#47)#52] -Results [5]: [i_product_name#40, i_brand#41, null AS i_class#53, null AS i_category#54, avg(qoh#47)#52 AS qoh#55] +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [avg(qoh#51)] +Aggregate Attributes [1]: [avg(qoh#51)#56] +Results [5]: [i_product_name#44, i_brand#45, null AS i_class#57, null AS i_category#58, avg(qoh#51)#56 AS qoh#59] (34) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61] +Output [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] (35) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61] -Keys [4]: [i_product_name#56, i_brand#57, i_class#58, i_category#59] -Functions [1]: [avg(inv_quantity_on_hand#62)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#62)#17] -Results [2]: [i_product_name#56, avg(inv_quantity_on_hand#62)#17 AS qoh#63] +Input [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] +Keys [4]: [i_product_name#60, i_brand#61, i_class#62, i_category#63] +Functions [1]: [avg(inv_quantity_on_hand#66)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#66)#17] +Results [2]: [i_product_name#60, avg(inv_quantity_on_hand#66)#17 AS qoh#67] (36) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#56, qoh#63] -Keys [1]: [i_product_name#56] -Functions [1]: [partial_avg(qoh#63)] -Aggregate Attributes [2]: [sum#64, count#65] -Results [3]: [i_product_name#56, sum#66, count#67] +Input [2]: [i_product_name#60, qoh#67] +Keys [1]: [i_product_name#60] +Functions [1]: [partial_avg(qoh#67)] +Aggregate Attributes [2]: [sum#68, count#69] +Results [3]: [i_product_name#60, sum#70, count#71] (37) Exchange -Input [3]: [i_product_name#56, sum#66, count#67] -Arguments: hashpartitioning(i_product_name#56, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [3]: [i_product_name#60, sum#70, count#71] +Arguments: hashpartitioning(i_product_name#60, 5), ENSURE_REQUIREMENTS, [plan_id=6] (38) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#56, sum#66, count#67] -Keys [1]: [i_product_name#56] -Functions [1]: [avg(qoh#63)] -Aggregate Attributes [1]: [avg(qoh#63)#68] -Results [5]: [i_product_name#56, null AS i_brand#69, null AS i_class#70, null AS i_category#71, avg(qoh#63)#68 AS qoh#72] +Input [3]: [i_product_name#60, sum#70, count#71] +Keys [1]: [i_product_name#60] +Functions [1]: [avg(qoh#67)] +Aggregate Attributes [1]: [avg(qoh#67)#72] +Results [5]: [i_product_name#60, null AS i_brand#73, null AS i_class#74, null AS i_category#75, avg(qoh#67)#72 AS qoh#76] (39) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78] +Output [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] (40) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78] -Keys [4]: [i_product_name#73, i_brand#74, i_class#75, i_category#76] -Functions [1]: [avg(inv_quantity_on_hand#79)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#79)#17] -Results [1]: [avg(inv_quantity_on_hand#79)#17 AS qoh#80] +Input [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] +Keys [4]: [i_product_name#77, i_brand#78, i_class#79, i_category#80] +Functions [1]: [avg(inv_quantity_on_hand#83)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#83)#17] +Results [1]: [avg(inv_quantity_on_hand#83)#17 AS qoh#84] (41) HashAggregate [codegen id : 28] -Input [1]: [qoh#80] +Input [1]: [qoh#84] Keys: [] -Functions [1]: [partial_avg(qoh#80)] -Aggregate Attributes [2]: [sum#81, count#82] -Results [2]: [sum#83, count#84] +Functions [1]: [partial_avg(qoh#84)] +Aggregate Attributes [2]: [sum#85, count#86] +Results [2]: [sum#87, count#88] (42) Exchange -Input [2]: [sum#83, count#84] +Input [2]: [sum#87, count#88] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] (43) HashAggregate [codegen id : 29] -Input [2]: [sum#83, count#84] +Input [2]: [sum#87, count#88] Keys: [] -Functions [1]: [avg(qoh#80)] -Aggregate Attributes [1]: [avg(qoh#80)#85] -Results [5]: [null AS i_product_name#86, null AS i_brand#87, null AS i_class#88, null AS i_category#89, avg(qoh#80)#85 AS qoh#90] +Functions [1]: [avg(qoh#84)] +Aggregate Attributes [1]: [avg(qoh#84)#89] +Results [5]: [null AS i_product_name#90, null AS i_brand#91, null AS i_class#92, null AS i_category#93, avg(qoh#84)#89 AS qoh#94] (44) Union (45) TakeOrderedAndProject -Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] -Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] +Input [5]: [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] +Arguments: 100, [qoh#28 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_class#26 ASC NULLS FIRST, i_category#27 ASC NULLS FIRST], [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] ===== Subqueries ===== @@ -291,22 +291,22 @@ BroadcastExchange (50) (46) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#91] +Output [2]: [d_date_sk#6, d_month_seq#95] 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), IsNotNull(d_date_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#91] +Input [2]: [d_date_sk#6, d_month_seq#95] (48) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#91] -Condition : (((isnotnull(d_month_seq#91) AND (d_month_seq#91 >= 1212)) AND (d_month_seq#91 <= 1223)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_month_seq#95] +Condition : (((isnotnull(d_month_seq#95) AND (d_month_seq#95 >= 1212)) AND (d_month_seq#95 <= 1223)) AND isnotnull(d_date_sk#6)) (49) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_month_seq#91] +Input [2]: [d_date_sk#6, d_month_seq#95] (50) BroadcastExchange Input [1]: [d_date_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index 22f73cc9b9db..d747066f5945 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter 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 f5458b1d362b..cbd381584558 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 @@ -211,74 +211,74 @@ Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_yea Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] -Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#18] +Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#30] (36) Exchange -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=7] (37) Sort [codegen id : 21] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 (38) Window -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#18 AS sum_sales#31, rn#30] -Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18, rn#30] +Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] (40) Exchange -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1), 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1), 5), ENSURE_REQUIREMENTS, [plan_id=8] (41) Sort [codegen id : 23] -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] -Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, (rn#30 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, (rn#31 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [5]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, rn#20] -Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1)] +Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] Join type: Inner Join condition: None (43) Project [codegen id : 24] -Output [10]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] +Output [10]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30] +Input [15]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] (44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] (45) Sort [codegen id : 33] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 (46) Window -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#18 AS sum_sales#39, rn#38] -Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18, rn#38] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] (48) Exchange -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] -Arguments: hashpartitioning(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1), 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Arguments: hashpartitioning(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1), 5), ENSURE_REQUIREMENTS, [plan_id=9] (49) Sort [codegen id : 35] -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] -Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, (rn#38 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, (rn#39 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [5]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, rn#20] -Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1)] +Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] Join type: Inner Join condition: None (51) Project [codegen id : 36] -Output [7]: [i_category#14, d_year#7, d_moy#8, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] +Output [7]: [i_category#14, d_year#7, d_moy#8, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 AS nsum#41] +Input [16]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] (52) TakeOrderedAndProject Input [7]: [i_category#14, d_year#7, d_moy#8, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] 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 b6e92af063df..f7859846a83d 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 @@ -184,66 +184,66 @@ Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_yea Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] -Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#18] +Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#30] (31) Exchange -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] (32) Sort [codegen id : 13] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 (33) Window -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#18 AS sum_sales#31, rn#30] -Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18, rn#30] +Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] (35) BroadcastExchange -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] 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), [plan_id=6] (36) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] -Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1)] +Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] Join type: Inner Join condition: None (37) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] (38) ReusedExchange [Reuses operator id: 31] -Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] (39) Sort [codegen id : 20] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 (40) Window -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#18 AS sum_sales#39, rn#38] -Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18, rn#38] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] (42) BroadcastExchange -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] 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), [plan_id=7] (43) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] -Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1)] +Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] Join type: Inner Join condition: None (44) Project [codegen id : 22] -Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] +Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 AS nsum#41] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] (45) TakeOrderedAndProject Input [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt index 221494d02593..8860d2b4473a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt @@ -142,51 +142,51 @@ Input [4]: [item_sk#10, d_date#6, sumws#11, rk#12] Arguments: [item_sk#10 ASC NULLS FIRST], false, 0 (16) ReusedExchange [Reuses operator id: 10] -Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +Output [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] (17) Sort [codegen id : 10] -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] +Arguments: [ws_item_sk#16 ASC NULLS FIRST, d_date#14 ASC NULLS FIRST], false, 0 (18) Window -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] +Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] +Arguments: [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] (19) Project [codegen id : 11] -Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] -Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] +Output [3]: [item_sk#13, sumws#15, rk#17] +Input [5]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17] (20) Exchange -Input [3]: [item_sk#16, sumws#17, rk#15] -Arguments: hashpartitioning(item_sk#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [3]: [item_sk#13, sumws#15, rk#17] +Arguments: hashpartitioning(item_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] (21) Sort [codegen id : 12] -Input [3]: [item_sk#16, sumws#17, rk#15] -Arguments: [item_sk#16 ASC NULLS FIRST], false, 0 +Input [3]: [item_sk#13, sumws#15, rk#17] +Arguments: [item_sk#13 ASC NULLS FIRST], false, 0 (22) SortMergeJoin [codegen id : 13] Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#16] +Right keys [1]: [item_sk#13] Join type: Inner -Join condition: (rk#12 >= rk#15) +Join condition: (rk#12 >= rk#17) (23) Project [codegen id : 13] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] -Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] +Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] +Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#13, sumws#15, rk#17] (24) HashAggregate [codegen id : 13] -Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] +Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [partial_sum(sumws#17)] +Functions [1]: [partial_sum(sumws#15)] Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] (25) HashAggregate [codegen id : 13] Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [sum(sumws#17)] -Aggregate Attributes [1]: [sum(sumws#17)#22] -Results [3]: [item_sk#10, d_date#6, sum(sumws#17)#22 AS cume_sales#23] +Functions [1]: [sum(sumws#15)] +Aggregate Attributes [1]: [sum(sumws#15)#22] +Results [3]: [item_sk#10, d_date#6, sum(sumws#15)#22 AS cume_sales#23] (26) Exchange Input [3]: [item_sk#10, d_date#6, cume_sales#23] @@ -267,51 +267,51 @@ Input [4]: [item_sk#32, d_date#28, sumss#33, rk#34] Arguments: [item_sk#32 ASC NULLS FIRST], false, 0 (43) ReusedExchange [Reuses operator id: 37] -Output [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] +Output [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] (44) Sort [codegen id : 24] -Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] -Arguments: [ss_item_sk#36 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +Arguments: [ss_item_sk#38 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST], false, 0 (45) Window -Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] -Arguments: [row_number() windowspecdefinition(ss_item_sk#36, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#37], [ss_item_sk#36], [d_date#35 ASC NULLS FIRST] +Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +Arguments: [row_number() windowspecdefinition(ss_item_sk#38, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#39], [ss_item_sk#38], [d_date#36 ASC NULLS FIRST] (46) Project [codegen id : 25] -Output [3]: [item_sk#32 AS item_sk#38, sumss#33 AS sumss#39, rk#37] -Input [5]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36, rk#37] +Output [3]: [item_sk#35, sumss#37, rk#39] +Input [5]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38, rk#39] (47) Exchange -Input [3]: [item_sk#38, sumss#39, rk#37] -Arguments: hashpartitioning(item_sk#38, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [3]: [item_sk#35, sumss#37, rk#39] +Arguments: hashpartitioning(item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=9] (48) Sort [codegen id : 26] -Input [3]: [item_sk#38, sumss#39, rk#37] -Arguments: [item_sk#38 ASC NULLS FIRST], false, 0 +Input [3]: [item_sk#35, sumss#37, rk#39] +Arguments: [item_sk#35 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 27] Left keys [1]: [item_sk#32] -Right keys [1]: [item_sk#38] +Right keys [1]: [item_sk#35] Join type: Inner -Join condition: (rk#34 >= rk#37) +Join condition: (rk#34 >= rk#39) (50) Project [codegen id : 27] -Output [4]: [item_sk#32, d_date#28, sumss#33, sumss#39] -Input [7]: [item_sk#32, d_date#28, sumss#33, rk#34, item_sk#38, sumss#39, rk#37] +Output [4]: [item_sk#32, d_date#28, sumss#33, sumss#37] +Input [7]: [item_sk#32, d_date#28, sumss#33, rk#34, item_sk#35, sumss#37, rk#39] (51) HashAggregate [codegen id : 27] -Input [4]: [item_sk#32, d_date#28, sumss#33, sumss#39] +Input [4]: [item_sk#32, d_date#28, sumss#33, sumss#37] Keys [3]: [item_sk#32, d_date#28, sumss#33] -Functions [1]: [partial_sum(sumss#39)] +Functions [1]: [partial_sum(sumss#37)] Aggregate Attributes [2]: [sum#40, isEmpty#41] Results [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] (52) HashAggregate [codegen id : 27] Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] Keys [3]: [item_sk#32, d_date#28, sumss#33] -Functions [1]: [sum(sumss#39)] -Aggregate Attributes [1]: [sum(sumss#39)#44] -Results [3]: [item_sk#32, d_date#28, sum(sumss#39)#44 AS cume_sales#45] +Functions [1]: [sum(sumss#37)] +Aggregate Attributes [1]: [sum(sumss#37)#44] +Results [3]: [item_sk#32, d_date#28, sum(sumss#37)#44 AS cume_sales#45] (53) Exchange Input [3]: [item_sk#32, d_date#28, cume_sales#45] @@ -348,51 +348,51 @@ Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [item_sk#46], [d_date#47 ASC NULLS FIRST] (61) ReusedExchange [Reuses operator id: 58] -Output [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Output [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] (62) Sort [codegen id : 60] -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] +Arguments: [item_sk#51 ASC NULLS FIRST, d_date#52 ASC NULLS FIRST], false, 0 (63) Window -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#46], [d_date#47 ASC NULLS FIRST] +Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] +Arguments: [row_number() windowspecdefinition(item_sk#51, d_date#52 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#55], [item_sk#51], [d_date#52 ASC NULLS FIRST] (64) Project [codegen id : 61] -Output [4]: [item_sk#46 AS item_sk#52, web_sales#48 AS web_sales#53, store_sales#49 AS store_sales#54, rk#51] -Input [5]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#51] +Output [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] +Input [5]: [item_sk#51, d_date#52, web_sales#53, store_sales#54, rk#55] (65) SortMergeJoin [codegen id : 62] Left keys [1]: [item_sk#46] -Right keys [1]: [item_sk#52] +Right keys [1]: [item_sk#51] Join type: Inner -Join condition: (rk#50 >= rk#51) +Join condition: (rk#50 >= rk#55) (66) Project [codegen id : 62] Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#52, web_sales#53, store_sales#54, rk#51] +Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#51, web_sales#53, store_sales#54, rk#55] (67) HashAggregate [codegen id : 62] Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] -Aggregate Attributes [2]: [max#55, max#56] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] +Aggregate Attributes [2]: [max#56, max#57] +Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] (68) HashAggregate [codegen id : 62] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Functions [2]: [max(web_sales#53), max(store_sales#54)] -Aggregate Attributes [2]: [max(web_sales#53)#59, max(store_sales#54)#60] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#59 AS web_cumulative#61, max(store_sales#54)#60 AS store_cumulative#62] +Aggregate Attributes [2]: [max(web_sales#53)#60, max(store_sales#54)#61] +Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#60 AS web_cumulative#62, max(store_sales#54)#61 AS store_cumulative#63] (69) Filter [codegen id : 62] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] -Condition : ((isnotnull(web_cumulative#61) AND isnotnull(store_cumulative#62)) AND (web_cumulative#61 > store_cumulative#62)) +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] +Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) (70) TakeOrderedAndProject -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] -Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] +Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] ===== Subqueries ===== @@ -405,22 +405,22 @@ BroadcastExchange (75) (71) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#63] +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#64] 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), IsNotNull(d_date_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#63] +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] (73) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#63] -Condition : (((isnotnull(d_month_seq#63) AND (d_month_seq#63 >= 1212)) AND (d_month_seq#63 <= 1223)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] +Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_month_seq#64 <= 1223)) AND isnotnull(d_date_sk#5)) (74) Project [codegen id : 1] Output [2]: [d_date_sk#5, d_date#6] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#63] +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] (75) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt index 051586e27057..dbb541f9eaa0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt @@ -131,38 +131,38 @@ Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] (14) ReusedExchange [Reuses operator id: 10] -Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +Output [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] (15) Sort [codegen id : 8] -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] +Arguments: [ws_item_sk#16 ASC NULLS FIRST, d_date#14 ASC NULLS FIRST], false, 0 (16) Window -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] +Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] +Arguments: [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] (17) Project [codegen id : 9] -Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] -Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] +Output [3]: [item_sk#13, sumws#15, rk#17] +Input [5]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17] (18) BroadcastExchange -Input [3]: [item_sk#16, sumws#17, rk#15] +Input [3]: [item_sk#13, sumws#15, rk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] (19) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#16] +Right keys [1]: [item_sk#13] Join type: Inner -Join condition: (rk#12 >= rk#15) +Join condition: (rk#12 >= rk#17) (20) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] -Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] +Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] +Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#13, sumws#15, rk#17] (21) HashAggregate [codegen id : 10] -Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] +Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [partial_sum(sumws#17)] +Functions [1]: [partial_sum(sumws#15)] Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] @@ -173,9 +173,9 @@ Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREME (23) HashAggregate [codegen id : 11] Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [sum(sumws#17)] -Aggregate Attributes [1]: [sum(sumws#17)#22] -Results [3]: [item_sk#10, d_date#6, sum(sumws#17)#22 AS cume_sales#23] +Functions [1]: [sum(sumws#15)] +Aggregate Attributes [1]: [sum(sumws#15)#22] +Results [3]: [item_sk#10, d_date#6, sum(sumws#15)#22 AS cume_sales#23] (24) Exchange Input [3]: [item_sk#10, d_date#6, cume_sales#23] @@ -248,38 +248,38 @@ Output [4]: [item_sk#32, d_date#28, sumss#33, rk#34] Input [5]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24, rk#34] (39) ReusedExchange [Reuses operator id: 35] -Output [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] +Output [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] (40) Sort [codegen id : 20] -Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] -Arguments: [ss_item_sk#36 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +Arguments: [ss_item_sk#38 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST], false, 0 (41) Window -Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] -Arguments: [row_number() windowspecdefinition(ss_item_sk#36, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#37], [ss_item_sk#36], [d_date#35 ASC NULLS FIRST] +Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +Arguments: [row_number() windowspecdefinition(ss_item_sk#38, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#39], [ss_item_sk#38], [d_date#36 ASC NULLS FIRST] (42) Project [codegen id : 21] -Output [3]: [item_sk#32 AS item_sk#38, sumss#33 AS sumss#39, rk#37] -Input [5]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36, rk#37] +Output [3]: [item_sk#35, sumss#37, rk#39] +Input [5]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38, rk#39] (43) BroadcastExchange -Input [3]: [item_sk#38, sumss#39, rk#37] +Input [3]: [item_sk#35, sumss#37, rk#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] (44) BroadcastHashJoin [codegen id : 22] Left keys [1]: [item_sk#32] -Right keys [1]: [item_sk#38] +Right keys [1]: [item_sk#35] Join type: Inner -Join condition: (rk#34 >= rk#37) +Join condition: (rk#34 >= rk#39) (45) Project [codegen id : 22] -Output [4]: [item_sk#32, d_date#28, sumss#33, sumss#39] -Input [7]: [item_sk#32, d_date#28, sumss#33, rk#34, item_sk#38, sumss#39, rk#37] +Output [4]: [item_sk#32, d_date#28, sumss#33, sumss#37] +Input [7]: [item_sk#32, d_date#28, sumss#33, rk#34, item_sk#35, sumss#37, rk#39] (46) HashAggregate [codegen id : 22] -Input [4]: [item_sk#32, d_date#28, sumss#33, sumss#39] +Input [4]: [item_sk#32, d_date#28, sumss#33, sumss#37] Keys [3]: [item_sk#32, d_date#28, sumss#33] -Functions [1]: [partial_sum(sumss#39)] +Functions [1]: [partial_sum(sumss#37)] Aggregate Attributes [2]: [sum#40, isEmpty#41] Results [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] @@ -290,9 +290,9 @@ Arguments: hashpartitioning(item_sk#32, d_date#28, sumss#33, 5), ENSURE_REQUIREM (48) HashAggregate [codegen id : 23] Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] Keys [3]: [item_sk#32, d_date#28, sumss#33] -Functions [1]: [sum(sumss#39)] -Aggregate Attributes [1]: [sum(sumss#39)#44] -Results [3]: [item_sk#32, d_date#28, sum(sumss#39)#44 AS cume_sales#45] +Functions [1]: [sum(sumss#37)] +Aggregate Attributes [1]: [sum(sumss#37)#44] +Results [3]: [item_sk#32, d_date#28, sum(sumss#37)#44 AS cume_sales#45] (49) Exchange Input [3]: [item_sk#32, d_date#28, cume_sales#45] @@ -329,55 +329,55 @@ Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [item_sk#46], [d_date#47 ASC NULLS FIRST] (57) ReusedExchange [Reuses operator id: 54] -Output [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Output [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] (58) Sort [codegen id : 52] -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] +Arguments: [item_sk#51 ASC NULLS FIRST, d_date#52 ASC NULLS FIRST], false, 0 (59) Window -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#46], [d_date#47 ASC NULLS FIRST] +Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] +Arguments: [row_number() windowspecdefinition(item_sk#51, d_date#52 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#55], [item_sk#51], [d_date#52 ASC NULLS FIRST] (60) Project [codegen id : 53] -Output [4]: [item_sk#46 AS item_sk#52, web_sales#48 AS web_sales#53, store_sales#49 AS store_sales#54, rk#51] -Input [5]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#51] +Output [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] +Input [5]: [item_sk#51, d_date#52, web_sales#53, store_sales#54, rk#55] (61) BroadcastExchange -Input [4]: [item_sk#52, web_sales#53, store_sales#54, rk#51] +Input [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] (62) BroadcastHashJoin [codegen id : 54] Left keys [1]: [item_sk#46] -Right keys [1]: [item_sk#52] +Right keys [1]: [item_sk#51] Join type: Inner -Join condition: (rk#50 >= rk#51) +Join condition: (rk#50 >= rk#55) (63) Project [codegen id : 54] Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#52, web_sales#53, store_sales#54, rk#51] +Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#51, web_sales#53, store_sales#54, rk#55] (64) HashAggregate [codegen id : 54] Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] -Aggregate Attributes [2]: [max#55, max#56] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] +Aggregate Attributes [2]: [max#56, max#57] +Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] (65) HashAggregate [codegen id : 54] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Functions [2]: [max(web_sales#53), max(store_sales#54)] -Aggregate Attributes [2]: [max(web_sales#53)#59, max(store_sales#54)#60] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#59 AS web_cumulative#61, max(store_sales#54)#60 AS store_cumulative#62] +Aggregate Attributes [2]: [max(web_sales#53)#60, max(store_sales#54)#61] +Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#60 AS web_cumulative#62, max(store_sales#54)#61 AS store_cumulative#63] (66) Filter [codegen id : 54] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] -Condition : ((isnotnull(web_cumulative#61) AND isnotnull(store_cumulative#62)) AND (web_cumulative#61 > store_cumulative#62)) +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] +Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) (67) TakeOrderedAndProject -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] -Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] +Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] ===== Subqueries ===== @@ -390,22 +390,22 @@ BroadcastExchange (72) (68) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#63] +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#64] 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), IsNotNull(d_date_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#63] +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] (70) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#63] -Condition : (((isnotnull(d_month_seq#63) AND (d_month_seq#63 >= 1212)) AND (d_month_seq#63 <= 1223)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] +Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_month_seq#64 <= 1223)) AND isnotnull(d_date_sk#5)) (71) Project [codegen id : 1] Output [2]: [d_date_sk#5, d_date#6] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#63] +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] (72) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] 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 e11a57f91afc..bf99a4902e45 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 @@ -211,74 +211,74 @@ Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] -Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#17] +Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#28] (36) Exchange -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=7] (37) Sort [codegen id : 21] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 (38) Window -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#17 AS sum_sales#29, rn#28] -Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17, rn#28] +Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] (40) Exchange -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, (rn#28 + 1), 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, (rn#29 + 1), 5), ENSURE_REQUIREMENTS, [plan_id=8] (41) Sort [codegen id : 23] -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] -Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, (rn#28 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, (rn#29 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [4]: [i_category#13, i_brand#12, cc_name#10, rn#19] -Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#28 + 1)] +Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#29 + 1)] Join type: Inner Join condition: None (43) Project [codegen id : 24] -Output [9]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] +Output [9]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] +Input [13]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] (44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] (45) Sort [codegen id : 33] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 (46) Window -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#17 AS sum_sales#36, rn#35] -Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] (48) Exchange -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] -Arguments: hashpartitioning(i_category#30, i_brand#31, cc_name#32, (rn#35 - 1), 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Arguments: hashpartitioning(i_category#30, i_brand#31, cc_name#32, (rn#36 - 1), 5), ENSURE_REQUIREMENTS, [plan_id=9] (49) Sort [codegen id : 35] -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, (rn#35 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, (rn#36 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [4]: [i_category#13, i_brand#12, cc_name#10, rn#19] -Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#35 - 1)] +Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#36 - 1)] Join type: Inner Join condition: None (51) Project [codegen id : 36] -Output [8]: [i_category#13, i_brand#12, d_year#7, d_moy#8, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] +Output [8]: [i_category#13, i_brand#12, d_year#7, d_moy#8, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 AS nsum#38] +Input [14]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] (52) TakeOrderedAndProject Input [8]: [i_category#13, i_brand#12, d_year#7, d_moy#8, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] 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 18537ebe2da3..8f3b905c0c08 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 @@ -184,66 +184,66 @@ Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] -Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#17] +Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#28] (31) Exchange -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] (32) Sort [codegen id : 13] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 (33) Window -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#17 AS sum_sales#29, rn#28] -Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17, rn#28] +Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] (35) BroadcastExchange -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] (36) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] -Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#28 + 1)] +Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#29 + 1)] Join type: Inner Join condition: None (37) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] (38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] (39) Sort [codegen id : 20] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 (40) Window -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#17 AS sum_sales#36, rn#35] -Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] (42) BroadcastExchange -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] (43) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] -Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#35 - 1)] +Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#36 - 1)] Join type: Inner Join condition: None (44) Project [codegen id : 22] -Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] +Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 AS nsum#38] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] (45) TakeOrderedAndProject Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt index 1b9d5f3f8077..fb185e813044 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt @@ -444,60 +444,60 @@ Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#41 Results [5]: [channel#37, id#38, cast(sum(sales#39)#139 as decimal(37,2)) AS sales#142, cast(sum(returns#40)#140 as decimal(37,2)) AS returns#143, cast(sum(profit#41)#141 as decimal(38,2)) AS profit#144] (76) ReusedExchange [Reuses operator id: 74] -Output [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Output [8]: [channel#145, id#146, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#147, isEmpty#148] (77) HashAggregate [codegen id : 48] -Input [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#41)#141] -Results [4]: [channel#37, sum(sales#39)#139 AS sales#145, sum(returns#40)#140 AS returns#146, sum(profit#41)#141 AS profit#147] +Input [8]: [channel#145, id#146, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#147, isEmpty#148] +Keys [2]: [channel#145, id#146] +Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#149)] +Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#149)#141] +Results [4]: [channel#145, sum(sales#39)#139 AS sales#150, sum(returns#40)#140 AS returns#151, sum(profit#149)#141 AS profit#152] (78) HashAggregate [codegen id : 48] -Input [4]: [channel#37, sales#145, returns#146, profit#147] -Keys [1]: [channel#37] -Functions [3]: [partial_sum(sales#145), partial_sum(returns#146), partial_sum(profit#147)] -Aggregate Attributes [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -Results [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] +Input [4]: [channel#145, sales#150, returns#151, profit#152] +Keys [1]: [channel#145] +Functions [3]: [partial_sum(sales#150), partial_sum(returns#151), partial_sum(profit#152)] +Aggregate Attributes [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Results [7]: [channel#145, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] (79) Exchange -Input [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] -Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [7]: [channel#145, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +Arguments: hashpartitioning(channel#145, 5), ENSURE_REQUIREMENTS, [plan_id=10] (80) HashAggregate [codegen id : 49] -Input [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] -Keys [1]: [channel#37] -Functions [3]: [sum(sales#145), sum(returns#146), sum(profit#147)] -Aggregate Attributes [3]: [sum(sales#145)#160, sum(returns#146)#161, sum(profit#147)#162] -Results [5]: [channel#37, null AS id#163, sum(sales#145)#160 AS sum(sales)#164, sum(returns#146)#161 AS sum(returns)#165, sum(profit#147)#162 AS sum(profit)#166] +Input [7]: [channel#145, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +Keys [1]: [channel#145] +Functions [3]: [sum(sales#150), sum(returns#151), sum(profit#152)] +Aggregate Attributes [3]: [sum(sales#150)#165, sum(returns#151)#166, sum(profit#152)#167] +Results [5]: [channel#145, null AS id#168, sum(sales#150)#165 AS sum(sales)#169, sum(returns#151)#166 AS sum(returns)#170, sum(profit#152)#167 AS sum(profit)#171] (81) ReusedExchange [Reuses operator id: 74] -Output [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Output [8]: [channel#172, id#173, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#174, isEmpty#175] (82) HashAggregate [codegen id : 73] -Input [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#41)#141] -Results [3]: [sum(sales#39)#139 AS sales#167, sum(returns#40)#140 AS returns#168, sum(profit#41)#141 AS profit#169] +Input [8]: [channel#172, id#173, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#174, isEmpty#175] +Keys [2]: [channel#172, id#173] +Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#176)] +Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#176)#141] +Results [3]: [sum(sales#39)#139 AS sales#177, sum(returns#40)#140 AS returns#178, sum(profit#176)#141 AS profit#179] (83) HashAggregate [codegen id : 73] -Input [3]: [sales#167, returns#168, profit#169] +Input [3]: [sales#177, returns#178, profit#179] Keys: [] -Functions [3]: [partial_sum(sales#167), partial_sum(returns#168), partial_sum(profit#169)] -Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] -Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Functions [3]: [partial_sum(sales#177), partial_sum(returns#178), partial_sum(profit#179)] +Aggregate Attributes [6]: [sum#180, isEmpty#181, sum#182, isEmpty#183, sum#184, isEmpty#185] +Results [6]: [sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] (84) Exchange -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [6]: [sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] (85) HashAggregate [codegen id : 74] -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [6]: [sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] Keys: [] -Functions [3]: [sum(sales#167), sum(returns#168), sum(profit#169)] -Aggregate Attributes [3]: [sum(sales#167)#182, sum(returns#168)#183, sum(profit#169)#184] -Results [5]: [null AS channel#185, null AS id#186, sum(sales#167)#182 AS sum(sales)#187, sum(returns#168)#183 AS sum(returns)#188, sum(profit#169)#184 AS sum(profit)#189] +Functions [3]: [sum(sales#177), sum(returns#178), sum(profit#179)] +Aggregate Attributes [3]: [sum(sales#177)#192, sum(returns#178)#193, sum(profit#179)#194] +Results [5]: [null AS channel#195, null AS id#196, sum(sales#177)#192 AS sum(sales)#197, sum(returns#178)#193 AS sum(returns)#198, sum(profit#179)#194 AS sum(profit)#199] (86) Union @@ -534,22 +534,22 @@ BroadcastExchange (95) (91) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_date#190] +Output [2]: [d_date_sk#24, d_date#200] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_date#190] +Input [2]: [d_date_sk#24, d_date#200] (93) Filter [codegen id : 1] -Input [2]: [d_date_sk#24, d_date#190] -Condition : (((isnotnull(d_date#190) AND (d_date#190 >= 1998-08-04)) AND (d_date#190 <= 1998-08-18)) AND isnotnull(d_date_sk#24)) +Input [2]: [d_date_sk#24, d_date#200] +Condition : (((isnotnull(d_date#200) AND (d_date#200 >= 1998-08-04)) AND (d_date#200 <= 1998-08-18)) AND isnotnull(d_date_sk#24)) (94) Project [codegen id : 1] Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_date#190] +Input [2]: [d_date_sk#24, d_date#200] (95) BroadcastExchange Input [1]: [d_date_sk#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 8d58ec413658..d93611ad5dc6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -429,60 +429,60 @@ Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#41 Results [5]: [channel#37, id#38, cast(sum(sales#39)#139 as decimal(37,2)) AS sales#142, cast(sum(returns#40)#140 as decimal(37,2)) AS returns#143, cast(sum(profit#41)#141 as decimal(38,2)) AS profit#144] (73) ReusedExchange [Reuses operator id: 71] -Output [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Output [8]: [channel#145, id#146, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#147, isEmpty#148] (74) HashAggregate [codegen id : 42] -Input [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#41)#141] -Results [4]: [channel#37, sum(sales#39)#139 AS sales#145, sum(returns#40)#140 AS returns#146, sum(profit#41)#141 AS profit#147] +Input [8]: [channel#145, id#146, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#147, isEmpty#148] +Keys [2]: [channel#145, id#146] +Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#149)] +Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#149)#141] +Results [4]: [channel#145, sum(sales#39)#139 AS sales#150, sum(returns#40)#140 AS returns#151, sum(profit#149)#141 AS profit#152] (75) HashAggregate [codegen id : 42] -Input [4]: [channel#37, sales#145, returns#146, profit#147] -Keys [1]: [channel#37] -Functions [3]: [partial_sum(sales#145), partial_sum(returns#146), partial_sum(profit#147)] -Aggregate Attributes [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -Results [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] +Input [4]: [channel#145, sales#150, returns#151, profit#152] +Keys [1]: [channel#145] +Functions [3]: [partial_sum(sales#150), partial_sum(returns#151), partial_sum(profit#152)] +Aggregate Attributes [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Results [7]: [channel#145, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] (76) Exchange -Input [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] -Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [7]: [channel#145, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +Arguments: hashpartitioning(channel#145, 5), ENSURE_REQUIREMENTS, [plan_id=9] (77) HashAggregate [codegen id : 43] -Input [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] -Keys [1]: [channel#37] -Functions [3]: [sum(sales#145), sum(returns#146), sum(profit#147)] -Aggregate Attributes [3]: [sum(sales#145)#160, sum(returns#146)#161, sum(profit#147)#162] -Results [5]: [channel#37, null AS id#163, sum(sales#145)#160 AS sum(sales)#164, sum(returns#146)#161 AS sum(returns)#165, sum(profit#147)#162 AS sum(profit)#166] +Input [7]: [channel#145, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +Keys [1]: [channel#145] +Functions [3]: [sum(sales#150), sum(returns#151), sum(profit#152)] +Aggregate Attributes [3]: [sum(sales#150)#165, sum(returns#151)#166, sum(profit#152)#167] +Results [5]: [channel#145, null AS id#168, sum(sales#150)#165 AS sum(sales)#169, sum(returns#151)#166 AS sum(returns)#170, sum(profit#152)#167 AS sum(profit)#171] (78) ReusedExchange [Reuses operator id: 71] -Output [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Output [8]: [channel#172, id#173, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#174, isEmpty#175] (79) HashAggregate [codegen id : 64] -Input [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#41)#141] -Results [3]: [sum(sales#39)#139 AS sales#167, sum(returns#40)#140 AS returns#168, sum(profit#41)#141 AS profit#169] +Input [8]: [channel#172, id#173, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#174, isEmpty#175] +Keys [2]: [channel#172, id#173] +Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#176)] +Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#176)#141] +Results [3]: [sum(sales#39)#139 AS sales#177, sum(returns#40)#140 AS returns#178, sum(profit#176)#141 AS profit#179] (80) HashAggregate [codegen id : 64] -Input [3]: [sales#167, returns#168, profit#169] +Input [3]: [sales#177, returns#178, profit#179] Keys: [] -Functions [3]: [partial_sum(sales#167), partial_sum(returns#168), partial_sum(profit#169)] -Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] -Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Functions [3]: [partial_sum(sales#177), partial_sum(returns#178), partial_sum(profit#179)] +Aggregate Attributes [6]: [sum#180, isEmpty#181, sum#182, isEmpty#183, sum#184, isEmpty#185] +Results [6]: [sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] (81) Exchange -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [6]: [sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] (82) HashAggregate [codegen id : 65] -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [6]: [sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] Keys: [] -Functions [3]: [sum(sales#167), sum(returns#168), sum(profit#169)] -Aggregate Attributes [3]: [sum(sales#167)#182, sum(returns#168)#183, sum(profit#169)#184] -Results [5]: [null AS channel#185, null AS id#186, sum(sales#167)#182 AS sum(sales)#187, sum(returns#168)#183 AS sum(returns)#188, sum(profit#169)#184 AS sum(profit)#189] +Functions [3]: [sum(sales#177), sum(returns#178), sum(profit#179)] +Aggregate Attributes [3]: [sum(sales#177)#192, sum(returns#178)#193, sum(profit#179)#194] +Results [5]: [null AS channel#195, null AS id#196, sum(sales#177)#192 AS sum(sales)#197, sum(returns#178)#193 AS sum(returns)#198, sum(profit#179)#194 AS sum(profit)#199] (83) Union @@ -519,22 +519,22 @@ BroadcastExchange (92) (88) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#190] +Output [2]: [d_date_sk#22, d_date#200] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct (89) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#190] +Input [2]: [d_date_sk#22, d_date#200] (90) Filter [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#190] -Condition : (((isnotnull(d_date#190) AND (d_date#190 >= 1998-08-04)) AND (d_date#190 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#22, d_date#200] +Condition : (((isnotnull(d_date#200) AND (d_date#200 >= 1998-08-04)) AND (d_date#200 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) (91) Project [codegen id : 1] Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_date#190] +Input [2]: [d_date_sk#22, d_date#200] (92) BroadcastExchange Input [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt index 9c28ff9f351d..a4c009f8219b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt @@ -186,265 +186,265 @@ Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23] +Results [9]: [i_category#16 AS i_category#23, i_class#15 AS i_class#24, i_brand#14 AS i_brand#25, i_product_name#17 AS i_product_name#26, d_year#8 AS d_year#27, d_qoy#10 AS d_qoy#28, d_moy#9 AS d_moy#29, s_store_id#12 AS s_store_id#30, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#31] (25) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33] +Output [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] (26) HashAggregate [codegen id : 16] -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33] -Keys [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31] -Functions [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22] -Results [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22 AS sumsales#36] +Input [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] +Keys [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39] +Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22] +Results [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22 AS sumsales#44] (27) HashAggregate [codegen id : 16] -Input [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sumsales#36] -Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30] -Functions [1]: [partial_sum(sumsales#36)] -Aggregate Attributes [2]: [sum#37, isEmpty#38] -Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] +Input [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sumsales#44] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [partial_sum(sumsales#44)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] (28) Exchange -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, 5), ENSURE_REQUIREMENTS, [plan_id=5] (29) HashAggregate [codegen id : 17] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] -Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30] -Functions [1]: [sum(sumsales#36)] -Aggregate Attributes [1]: [sum(sumsales#36)#41] -Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, null AS s_store_id#42, sum(sumsales#36)#41 AS sumsales#43] +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [sum(sumsales#44)] +Aggregate Attributes [1]: [sum(sumsales#44)#49] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] (30) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53] +Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] (31) HashAggregate [codegen id : 25] -Input [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53] -Keys [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51] -Functions [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22] -Results [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22 AS sumsales#56] +Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] +Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59] +Functions [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22] +Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22 AS sumsales#64] (32) HashAggregate [codegen id : 25] -Input [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sumsales#56] -Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49] -Functions [1]: [partial_sum(sumsales#56)] -Aggregate Attributes [2]: [sum#57, isEmpty#58] -Results [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] +Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#64] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [partial_sum(sumsales#64)] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] (33) Exchange -Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] -Arguments: hashpartitioning(i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [plan_id=6] (34) HashAggregate [codegen id : 26] -Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] -Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49] -Functions [1]: [sum(sumsales#56)] -Aggregate Attributes [1]: [sum(sumsales#56)#61] -Results [9]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, null AS d_moy#62, null AS s_store_id#63, sum(sumsales#56)#61 AS sumsales#64] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [sum(sumsales#64)] +Aggregate Attributes [1]: [sum(sumsales#64)#69] +Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#64)#69 AS sumsales#72] (35) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74] +Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] (36) HashAggregate [codegen id : 34] -Input [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74] -Keys [8]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72] -Functions [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22] -Results [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22 AS sumsales#77] +Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] +Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80] +Functions [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22] +Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22 AS sumsales#85] (37) HashAggregate [codegen id : 34] -Input [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sumsales#77] -Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69] -Functions [1]: [partial_sum(sumsales#77)] -Aggregate Attributes [2]: [sum#78, isEmpty#79] -Results [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] +Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#85] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [partial_sum(sumsales#85)] +Aggregate Attributes [2]: [sum#86, isEmpty#87] +Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] (38) Exchange -Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] -Arguments: hashpartitioning(i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=7] (39) HashAggregate [codegen id : 35] -Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] -Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69] -Functions [1]: [sum(sumsales#77)] -Aggregate Attributes [1]: [sum(sumsales#77)#82] -Results [9]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, null AS d_qoy#83, null AS d_moy#84, null AS s_store_id#85, sum(sumsales#77)#82 AS sumsales#86] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [sum(sumsales#85)] +Aggregate Attributes [1]: [sum(sumsales#85)#90] +Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#85)#90 AS sumsales#94] (40) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96] +Output [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] (41) HashAggregate [codegen id : 43] -Input [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96] -Keys [8]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94] -Functions [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22] -Results [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22 AS sumsales#99] +Input [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] +Keys [8]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102] +Functions [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22] +Results [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22 AS sumsales#107] (42) HashAggregate [codegen id : 43] -Input [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sumsales#99] -Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90] -Functions [1]: [partial_sum(sumsales#99)] -Aggregate Attributes [2]: [sum#100, isEmpty#101] -Results [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] +Input [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sumsales#107] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [partial_sum(sumsales#107)] +Aggregate Attributes [2]: [sum#108, isEmpty#109] +Results [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] (43) Exchange -Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] -Arguments: hashpartitioning(i_category#87, i_class#88, i_brand#89, i_product_name#90, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Arguments: hashpartitioning(i_category#95, i_class#96, i_brand#97, i_product_name#98, 5), ENSURE_REQUIREMENTS, [plan_id=8] (44) HashAggregate [codegen id : 44] -Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] -Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90] -Functions [1]: [sum(sumsales#99)] -Aggregate Attributes [1]: [sum(sumsales#99)#104] -Results [9]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, null AS d_year#105, null AS d_qoy#106, null AS d_moy#107, null AS s_store_id#108, sum(sumsales#99)#104 AS sumsales#109] +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [sum(sumsales#107)] +Aggregate Attributes [1]: [sum(sumsales#107)#112] +Results [9]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, null AS d_year#113, null AS d_qoy#114, null AS d_moy#115, null AS s_store_id#116, sum(sumsales#107)#112 AS sumsales#117] (45) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119] +Output [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] (46) HashAggregate [codegen id : 52] -Input [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119] -Keys [8]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117] -Functions [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22] -Results [4]: [i_category#110, i_class#111, i_brand#112, sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22 AS sumsales#122] +Input [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] +Keys [8]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125] +Functions [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22] +Results [4]: [i_category#118, i_class#119, i_brand#120, sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22 AS sumsales#130] (47) HashAggregate [codegen id : 52] -Input [4]: [i_category#110, i_class#111, i_brand#112, sumsales#122] -Keys [3]: [i_category#110, i_class#111, i_brand#112] -Functions [1]: [partial_sum(sumsales#122)] -Aggregate Attributes [2]: [sum#123, isEmpty#124] -Results [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] +Input [4]: [i_category#118, i_class#119, i_brand#120, sumsales#130] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [partial_sum(sumsales#130)] +Aggregate Attributes [2]: [sum#131, isEmpty#132] +Results [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] (48) Exchange -Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] -Arguments: hashpartitioning(i_category#110, i_class#111, i_brand#112, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Arguments: hashpartitioning(i_category#118, i_class#119, i_brand#120, 5), ENSURE_REQUIREMENTS, [plan_id=9] (49) HashAggregate [codegen id : 53] -Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] -Keys [3]: [i_category#110, i_class#111, i_brand#112] -Functions [1]: [sum(sumsales#122)] -Aggregate Attributes [1]: [sum(sumsales#122)#127] -Results [9]: [i_category#110, i_class#111, i_brand#112, null AS i_product_name#128, null AS d_year#129, null AS d_qoy#130, null AS d_moy#131, null AS s_store_id#132, sum(sumsales#122)#127 AS sumsales#133] +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [sum(sumsales#130)] +Aggregate Attributes [1]: [sum(sumsales#130)#135] +Results [9]: [i_category#118, i_class#119, i_brand#120, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#130)#135 AS sumsales#141] (50) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143] +Output [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] (51) HashAggregate [codegen id : 61] -Input [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143] -Keys [8]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141] -Functions [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22] -Results [3]: [i_category#134, i_class#135, sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22 AS sumsales#146] +Input [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] +Keys [8]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149] +Functions [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22] +Results [3]: [i_category#142, i_class#143, sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22 AS sumsales#154] (52) HashAggregate [codegen id : 61] -Input [3]: [i_category#134, i_class#135, sumsales#146] -Keys [2]: [i_category#134, i_class#135] -Functions [1]: [partial_sum(sumsales#146)] -Aggregate Attributes [2]: [sum#147, isEmpty#148] -Results [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] +Input [3]: [i_category#142, i_class#143, sumsales#154] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [partial_sum(sumsales#154)] +Aggregate Attributes [2]: [sum#155, isEmpty#156] +Results [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] (53) Exchange -Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] -Arguments: hashpartitioning(i_category#134, i_class#135, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Arguments: hashpartitioning(i_category#142, i_class#143, 5), ENSURE_REQUIREMENTS, [plan_id=10] (54) HashAggregate [codegen id : 62] -Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] -Keys [2]: [i_category#134, i_class#135] -Functions [1]: [sum(sumsales#146)] -Aggregate Attributes [1]: [sum(sumsales#146)#151] -Results [9]: [i_category#134, i_class#135, null AS i_brand#152, null AS i_product_name#153, null AS d_year#154, null AS d_qoy#155, null AS d_moy#156, null AS s_store_id#157, sum(sumsales#146)#151 AS sumsales#158] +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [sum(sumsales#154)] +Aggregate Attributes [1]: [sum(sumsales#154)#159] +Results [9]: [i_category#142, i_class#143, null AS i_brand#160, null AS i_product_name#161, null AS d_year#162, null AS d_qoy#163, null AS d_moy#164, null AS s_store_id#165, sum(sumsales#154)#159 AS sumsales#166] (55) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168] +Output [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] (56) HashAggregate [codegen id : 70] -Input [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168] -Keys [8]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166] -Functions [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22] -Results [2]: [i_category#159, sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22 AS sumsales#171] +Input [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] +Keys [8]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174] +Functions [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22] +Results [2]: [i_category#167, sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22 AS sumsales#179] (57) HashAggregate [codegen id : 70] -Input [2]: [i_category#159, sumsales#171] -Keys [1]: [i_category#159] -Functions [1]: [partial_sum(sumsales#171)] -Aggregate Attributes [2]: [sum#172, isEmpty#173] -Results [3]: [i_category#159, sum#174, isEmpty#175] +Input [2]: [i_category#167, sumsales#179] +Keys [1]: [i_category#167] +Functions [1]: [partial_sum(sumsales#179)] +Aggregate Attributes [2]: [sum#180, isEmpty#181] +Results [3]: [i_category#167, sum#182, isEmpty#183] (58) Exchange -Input [3]: [i_category#159, sum#174, isEmpty#175] -Arguments: hashpartitioning(i_category#159, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Input [3]: [i_category#167, sum#182, isEmpty#183] +Arguments: hashpartitioning(i_category#167, 5), ENSURE_REQUIREMENTS, [plan_id=11] (59) HashAggregate [codegen id : 71] -Input [3]: [i_category#159, sum#174, isEmpty#175] -Keys [1]: [i_category#159] -Functions [1]: [sum(sumsales#171)] -Aggregate Attributes [1]: [sum(sumsales#171)#176] -Results [9]: [i_category#159, null AS i_class#177, null AS i_brand#178, null AS i_product_name#179, null AS d_year#180, null AS d_qoy#181, null AS d_moy#182, null AS s_store_id#183, sum(sumsales#171)#176 AS sumsales#184] +Input [3]: [i_category#167, sum#182, isEmpty#183] +Keys [1]: [i_category#167] +Functions [1]: [sum(sumsales#179)] +Aggregate Attributes [1]: [sum(sumsales#179)#184] +Results [9]: [i_category#167, null AS i_class#185, null AS i_brand#186, null AS i_product_name#187, null AS d_year#188, null AS d_qoy#189, null AS d_moy#190, null AS s_store_id#191, sum(sumsales#179)#184 AS sumsales#192] (60) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194] +Output [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] (61) HashAggregate [codegen id : 79] -Input [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194] -Keys [8]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192] -Functions [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22] -Results [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22 AS sumsales#197] +Input [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] +Keys [8]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200] +Functions [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22] +Results [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22 AS sumsales#205] (62) HashAggregate [codegen id : 79] -Input [1]: [sumsales#197] +Input [1]: [sumsales#205] Keys: [] -Functions [1]: [partial_sum(sumsales#197)] -Aggregate Attributes [2]: [sum#198, isEmpty#199] -Results [2]: [sum#200, isEmpty#201] +Functions [1]: [partial_sum(sumsales#205)] +Aggregate Attributes [2]: [sum#206, isEmpty#207] +Results [2]: [sum#208, isEmpty#209] (63) Exchange -Input [2]: [sum#200, isEmpty#201] +Input [2]: [sum#208, isEmpty#209] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] (64) HashAggregate [codegen id : 80] -Input [2]: [sum#200, isEmpty#201] +Input [2]: [sum#208, isEmpty#209] Keys: [] -Functions [1]: [sum(sumsales#197)] -Aggregate Attributes [1]: [sum(sumsales#197)#202] -Results [9]: [null AS i_category#203, null AS i_class#204, null AS i_brand#205, null AS i_product_name#206, null AS d_year#207, null AS d_qoy#208, null AS d_moy#209, null AS s_store_id#210, sum(sumsales#197)#202 AS sumsales#211] +Functions [1]: [sum(sumsales#205)] +Aggregate Attributes [1]: [sum(sumsales#205)#210] +Results [9]: [null AS i_category#211, null AS i_class#212, null AS i_brand#213, null AS i_product_name#214, null AS d_year#215, null AS d_qoy#216, null AS d_moy#217, null AS s_store_id#218, sum(sumsales#205)#210 AS sumsales#219] (65) Union (66) Sort [codegen id : 81] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 (67) WindowGroupLimit -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Partial +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Partial (68) Exchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, [plan_id=13] (69) Sort [codegen id : 82] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 (70) WindowGroupLimit -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Final +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Final (71) Window -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#212], [i_category#16], [sumsales#23 DESC NULLS LAST] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [rank(sumsales#31) windowspecdefinition(i_category#23, sumsales#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#220], [i_category#23], [sumsales#31 DESC NULLS LAST] (72) Filter [codegen id : 83] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] -Condition : (rk#212 <= 100) +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Condition : (rk#220 <= 100) (73) TakeOrderedAndProject -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] -Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#212 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_product_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_qoy#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, sumsales#31 ASC NULLS FIRST, rk#220 ASC NULLS FIRST], [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] ===== Subqueries ===== @@ -457,22 +457,22 @@ BroadcastExchange (78) (74) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Output [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] 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), IsNotNull(d_date_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] (76) Filter [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#213) AND (d_month_seq#213 >= 1212)) AND (d_month_seq#213 <= 1223)) AND isnotnull(d_date_sk#7)) +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#221) AND (d_month_seq#221 >= 1212)) AND (d_month_seq#221 <= 1223)) AND isnotnull(d_date_sk#7)) (77) Project [codegen id : 1] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] (78) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt index 795fa297b9ba..b6a4358c4d43 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter Union WholeStageCodegen (8) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 WholeStageCodegen (7) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index 75d526da4ba7..417af4fe924e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -171,265 +171,265 @@ Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23] +Results [9]: [i_category#16 AS i_category#23, i_class#15 AS i_class#24, i_brand#14 AS i_brand#25, i_product_name#17 AS i_product_name#26, d_year#8 AS d_year#27, d_qoy#10 AS d_qoy#28, d_moy#9 AS d_moy#29, s_store_id#12 AS s_store_id#30, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#31] (22) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33] +Output [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] (23) HashAggregate [codegen id : 10] -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33] -Keys [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31] -Functions [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22] -Results [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22 AS sumsales#36] +Input [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] +Keys [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39] +Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22] +Results [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22 AS sumsales#44] (24) HashAggregate [codegen id : 10] -Input [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sumsales#36] -Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30] -Functions [1]: [partial_sum(sumsales#36)] -Aggregate Attributes [2]: [sum#37, isEmpty#38] -Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] +Input [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sumsales#44] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [partial_sum(sumsales#44)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] (25) Exchange -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, 5), ENSURE_REQUIREMENTS, [plan_id=4] (26) HashAggregate [codegen id : 11] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] -Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30] -Functions [1]: [sum(sumsales#36)] -Aggregate Attributes [1]: [sum(sumsales#36)#41] -Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, null AS s_store_id#42, sum(sumsales#36)#41 AS sumsales#43] +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [sum(sumsales#44)] +Aggregate Attributes [1]: [sum(sumsales#44)#49] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] (27) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53] +Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] (28) HashAggregate [codegen id : 16] -Input [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53] -Keys [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51] -Functions [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22] -Results [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22 AS sumsales#56] +Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] +Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59] +Functions [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22] +Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22 AS sumsales#64] (29) HashAggregate [codegen id : 16] -Input [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sumsales#56] -Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49] -Functions [1]: [partial_sum(sumsales#56)] -Aggregate Attributes [2]: [sum#57, isEmpty#58] -Results [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] +Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#64] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [partial_sum(sumsales#64)] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] (30) Exchange -Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] -Arguments: hashpartitioning(i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [plan_id=5] (31) HashAggregate [codegen id : 17] -Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] -Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49] -Functions [1]: [sum(sumsales#56)] -Aggregate Attributes [1]: [sum(sumsales#56)#61] -Results [9]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, null AS d_moy#62, null AS s_store_id#63, sum(sumsales#56)#61 AS sumsales#64] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [sum(sumsales#64)] +Aggregate Attributes [1]: [sum(sumsales#64)#69] +Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#64)#69 AS sumsales#72] (32) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74] +Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] (33) HashAggregate [codegen id : 22] -Input [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74] -Keys [8]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72] -Functions [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22] -Results [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22 AS sumsales#77] +Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] +Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80] +Functions [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22] +Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22 AS sumsales#85] (34) HashAggregate [codegen id : 22] -Input [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sumsales#77] -Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69] -Functions [1]: [partial_sum(sumsales#77)] -Aggregate Attributes [2]: [sum#78, isEmpty#79] -Results [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] +Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#85] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [partial_sum(sumsales#85)] +Aggregate Attributes [2]: [sum#86, isEmpty#87] +Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] (35) Exchange -Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] -Arguments: hashpartitioning(i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=6] (36) HashAggregate [codegen id : 23] -Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] -Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69] -Functions [1]: [sum(sumsales#77)] -Aggregate Attributes [1]: [sum(sumsales#77)#82] -Results [9]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, null AS d_qoy#83, null AS d_moy#84, null AS s_store_id#85, sum(sumsales#77)#82 AS sumsales#86] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [sum(sumsales#85)] +Aggregate Attributes [1]: [sum(sumsales#85)#90] +Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#85)#90 AS sumsales#94] (37) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96] +Output [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] (38) HashAggregate [codegen id : 28] -Input [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96] -Keys [8]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94] -Functions [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22] -Results [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22 AS sumsales#99] +Input [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] +Keys [8]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102] +Functions [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22] +Results [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22 AS sumsales#107] (39) HashAggregate [codegen id : 28] -Input [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sumsales#99] -Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90] -Functions [1]: [partial_sum(sumsales#99)] -Aggregate Attributes [2]: [sum#100, isEmpty#101] -Results [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] +Input [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sumsales#107] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [partial_sum(sumsales#107)] +Aggregate Attributes [2]: [sum#108, isEmpty#109] +Results [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] (40) Exchange -Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] -Arguments: hashpartitioning(i_category#87, i_class#88, i_brand#89, i_product_name#90, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Arguments: hashpartitioning(i_category#95, i_class#96, i_brand#97, i_product_name#98, 5), ENSURE_REQUIREMENTS, [plan_id=7] (41) HashAggregate [codegen id : 29] -Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] -Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90] -Functions [1]: [sum(sumsales#99)] -Aggregate Attributes [1]: [sum(sumsales#99)#104] -Results [9]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, null AS d_year#105, null AS d_qoy#106, null AS d_moy#107, null AS s_store_id#108, sum(sumsales#99)#104 AS sumsales#109] +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [sum(sumsales#107)] +Aggregate Attributes [1]: [sum(sumsales#107)#112] +Results [9]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, null AS d_year#113, null AS d_qoy#114, null AS d_moy#115, null AS s_store_id#116, sum(sumsales#107)#112 AS sumsales#117] (42) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119] +Output [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] (43) HashAggregate [codegen id : 34] -Input [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119] -Keys [8]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117] -Functions [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22] -Results [4]: [i_category#110, i_class#111, i_brand#112, sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22 AS sumsales#122] +Input [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] +Keys [8]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125] +Functions [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22] +Results [4]: [i_category#118, i_class#119, i_brand#120, sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22 AS sumsales#130] (44) HashAggregate [codegen id : 34] -Input [4]: [i_category#110, i_class#111, i_brand#112, sumsales#122] -Keys [3]: [i_category#110, i_class#111, i_brand#112] -Functions [1]: [partial_sum(sumsales#122)] -Aggregate Attributes [2]: [sum#123, isEmpty#124] -Results [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] +Input [4]: [i_category#118, i_class#119, i_brand#120, sumsales#130] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [partial_sum(sumsales#130)] +Aggregate Attributes [2]: [sum#131, isEmpty#132] +Results [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] (45) Exchange -Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] -Arguments: hashpartitioning(i_category#110, i_class#111, i_brand#112, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Arguments: hashpartitioning(i_category#118, i_class#119, i_brand#120, 5), ENSURE_REQUIREMENTS, [plan_id=8] (46) HashAggregate [codegen id : 35] -Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] -Keys [3]: [i_category#110, i_class#111, i_brand#112] -Functions [1]: [sum(sumsales#122)] -Aggregate Attributes [1]: [sum(sumsales#122)#127] -Results [9]: [i_category#110, i_class#111, i_brand#112, null AS i_product_name#128, null AS d_year#129, null AS d_qoy#130, null AS d_moy#131, null AS s_store_id#132, sum(sumsales#122)#127 AS sumsales#133] +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [sum(sumsales#130)] +Aggregate Attributes [1]: [sum(sumsales#130)#135] +Results [9]: [i_category#118, i_class#119, i_brand#120, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#130)#135 AS sumsales#141] (47) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143] +Output [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] (48) HashAggregate [codegen id : 40] -Input [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143] -Keys [8]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141] -Functions [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22] -Results [3]: [i_category#134, i_class#135, sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22 AS sumsales#146] +Input [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] +Keys [8]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149] +Functions [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22] +Results [3]: [i_category#142, i_class#143, sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22 AS sumsales#154] (49) HashAggregate [codegen id : 40] -Input [3]: [i_category#134, i_class#135, sumsales#146] -Keys [2]: [i_category#134, i_class#135] -Functions [1]: [partial_sum(sumsales#146)] -Aggregate Attributes [2]: [sum#147, isEmpty#148] -Results [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] +Input [3]: [i_category#142, i_class#143, sumsales#154] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [partial_sum(sumsales#154)] +Aggregate Attributes [2]: [sum#155, isEmpty#156] +Results [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] (50) Exchange -Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] -Arguments: hashpartitioning(i_category#134, i_class#135, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Arguments: hashpartitioning(i_category#142, i_class#143, 5), ENSURE_REQUIREMENTS, [plan_id=9] (51) HashAggregate [codegen id : 41] -Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] -Keys [2]: [i_category#134, i_class#135] -Functions [1]: [sum(sumsales#146)] -Aggregate Attributes [1]: [sum(sumsales#146)#151] -Results [9]: [i_category#134, i_class#135, null AS i_brand#152, null AS i_product_name#153, null AS d_year#154, null AS d_qoy#155, null AS d_moy#156, null AS s_store_id#157, sum(sumsales#146)#151 AS sumsales#158] +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [sum(sumsales#154)] +Aggregate Attributes [1]: [sum(sumsales#154)#159] +Results [9]: [i_category#142, i_class#143, null AS i_brand#160, null AS i_product_name#161, null AS d_year#162, null AS d_qoy#163, null AS d_moy#164, null AS s_store_id#165, sum(sumsales#154)#159 AS sumsales#166] (52) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168] +Output [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] (53) HashAggregate [codegen id : 46] -Input [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168] -Keys [8]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166] -Functions [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22] -Results [2]: [i_category#159, sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22 AS sumsales#171] +Input [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] +Keys [8]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174] +Functions [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22] +Results [2]: [i_category#167, sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22 AS sumsales#179] (54) HashAggregate [codegen id : 46] -Input [2]: [i_category#159, sumsales#171] -Keys [1]: [i_category#159] -Functions [1]: [partial_sum(sumsales#171)] -Aggregate Attributes [2]: [sum#172, isEmpty#173] -Results [3]: [i_category#159, sum#174, isEmpty#175] +Input [2]: [i_category#167, sumsales#179] +Keys [1]: [i_category#167] +Functions [1]: [partial_sum(sumsales#179)] +Aggregate Attributes [2]: [sum#180, isEmpty#181] +Results [3]: [i_category#167, sum#182, isEmpty#183] (55) Exchange -Input [3]: [i_category#159, sum#174, isEmpty#175] -Arguments: hashpartitioning(i_category#159, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [3]: [i_category#167, sum#182, isEmpty#183] +Arguments: hashpartitioning(i_category#167, 5), ENSURE_REQUIREMENTS, [plan_id=10] (56) HashAggregate [codegen id : 47] -Input [3]: [i_category#159, sum#174, isEmpty#175] -Keys [1]: [i_category#159] -Functions [1]: [sum(sumsales#171)] -Aggregate Attributes [1]: [sum(sumsales#171)#176] -Results [9]: [i_category#159, null AS i_class#177, null AS i_brand#178, null AS i_product_name#179, null AS d_year#180, null AS d_qoy#181, null AS d_moy#182, null AS s_store_id#183, sum(sumsales#171)#176 AS sumsales#184] +Input [3]: [i_category#167, sum#182, isEmpty#183] +Keys [1]: [i_category#167] +Functions [1]: [sum(sumsales#179)] +Aggregate Attributes [1]: [sum(sumsales#179)#184] +Results [9]: [i_category#167, null AS i_class#185, null AS i_brand#186, null AS i_product_name#187, null AS d_year#188, null AS d_qoy#189, null AS d_moy#190, null AS s_store_id#191, sum(sumsales#179)#184 AS sumsales#192] (57) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194] +Output [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] (58) HashAggregate [codegen id : 52] -Input [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194] -Keys [8]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192] -Functions [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22] -Results [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22 AS sumsales#197] +Input [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] +Keys [8]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200] +Functions [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22] +Results [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22 AS sumsales#205] (59) HashAggregate [codegen id : 52] -Input [1]: [sumsales#197] +Input [1]: [sumsales#205] Keys: [] -Functions [1]: [partial_sum(sumsales#197)] -Aggregate Attributes [2]: [sum#198, isEmpty#199] -Results [2]: [sum#200, isEmpty#201] +Functions [1]: [partial_sum(sumsales#205)] +Aggregate Attributes [2]: [sum#206, isEmpty#207] +Results [2]: [sum#208, isEmpty#209] (60) Exchange -Input [2]: [sum#200, isEmpty#201] +Input [2]: [sum#208, isEmpty#209] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] (61) HashAggregate [codegen id : 53] -Input [2]: [sum#200, isEmpty#201] +Input [2]: [sum#208, isEmpty#209] Keys: [] -Functions [1]: [sum(sumsales#197)] -Aggregate Attributes [1]: [sum(sumsales#197)#202] -Results [9]: [null AS i_category#203, null AS i_class#204, null AS i_brand#205, null AS i_product_name#206, null AS d_year#207, null AS d_qoy#208, null AS d_moy#209, null AS s_store_id#210, sum(sumsales#197)#202 AS sumsales#211] +Functions [1]: [sum(sumsales#205)] +Aggregate Attributes [1]: [sum(sumsales#205)#210] +Results [9]: [null AS i_category#211, null AS i_class#212, null AS i_brand#213, null AS i_product_name#214, null AS d_year#215, null AS d_qoy#216, null AS d_moy#217, null AS s_store_id#218, sum(sumsales#205)#210 AS sumsales#219] (62) Union (63) Sort [codegen id : 54] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 (64) WindowGroupLimit -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Partial +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Partial (65) Exchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, [plan_id=12] (66) Sort [codegen id : 55] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 (67) WindowGroupLimit -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Final +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Final (68) Window -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#212], [i_category#16], [sumsales#23 DESC NULLS LAST] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [rank(sumsales#31) windowspecdefinition(i_category#23, sumsales#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#220], [i_category#23], [sumsales#31 DESC NULLS LAST] (69) Filter [codegen id : 56] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] -Condition : (rk#212 <= 100) +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Condition : (rk#220 <= 100) (70) TakeOrderedAndProject -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] -Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#212 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_product_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_qoy#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, sumsales#31 ASC NULLS FIRST, rk#220 ASC NULLS FIRST], [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] ===== Subqueries ===== @@ -442,22 +442,22 @@ BroadcastExchange (75) (71) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Output [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] 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), IsNotNull(d_date_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] (73) Filter [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#213) AND (d_month_seq#213 >= 1212)) AND (d_month_seq#213 <= 1223)) AND isnotnull(d_date_sk#7)) +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#221) AND (d_month_seq#221 >= 1212)) AND (d_month_seq#221 <= 1223)) AND isnotnull(d_date_sk#7)) (74) Project [codegen id : 1] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] (75) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index 89393f265a49..5a43dced056b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter Union WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 WholeStageCodegen (4) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt index 65f1a88c277b..8fcd79ff1552 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt @@ -517,204 +517,204 @@ Join type: LeftOuter Join condition: None (86) Project [codegen id : 32] -Output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#20, (cs_ext_sales_price#71 - coalesce(cr_return_amount#84, 0.00)) AS sales_amt#21] +Output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#85, (cs_ext_sales_price#71 - coalesce(cr_return_amount#84, 0.00)) AS sales_amt#86] Input [13]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80, cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] (87) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#89), dynamicpruningexpression(ss_sold_date_sk#89 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (88) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] (89) Filter [codegen id : 35] -Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Condition : isnotnull(ss_item_sk#85) +Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +Condition : isnotnull(ss_item_sk#87) (90) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94] +Output [5]: [i_item_sk#92, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96] (91) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#85] -Right keys [1]: [i_item_sk#90] +Left keys [1]: [ss_item_sk#87] +Right keys [1]: [i_item_sk#92] Join type: Inner Join condition: None (92) Project [codegen id : 35] -Output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94] -Input [10]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94] +Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96] +Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#92, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96] (93) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#95, d_year#96] +Output [2]: [d_date_sk#97, d_year#98] (94) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#89] -Right keys [1]: [d_date_sk#95] +Left keys [1]: [ss_sold_date_sk#91] +Right keys [1]: [d_date_sk#97] Join type: Inner Join condition: None (95) Project [codegen id : 35] -Output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96] -Input [11]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_date_sk#95, d_year#96] +Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98] +Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_date_sk#97, d_year#98] (96) Exchange -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96] -Arguments: hashpartitioning(ss_ticket_number#86, ss_item_sk#85, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98] +Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=12] (97) Sort [codegen id : 36] -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96] -Arguments: [ss_ticket_number#86 ASC NULLS FIRST, ss_item_sk#85 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98] +Arguments: [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST], false, 0 (98) ReusedExchange [Reuses operator id: 39] -Output [4]: [sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100] +Output [4]: [sr_item_sk#99, sr_ticket_number#100, sr_return_quantity#101, sr_return_amt#102] (99) Sort [codegen id : 38] -Input [4]: [sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100] -Arguments: [sr_ticket_number#98 ASC NULLS FIRST, sr_item_sk#97 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#99, sr_ticket_number#100, sr_return_quantity#101, sr_return_amt#102] +Arguments: [sr_ticket_number#100 ASC NULLS FIRST, sr_item_sk#99 ASC NULLS FIRST], false, 0 (100) SortMergeJoin [codegen id : 39] -Left keys [2]: [ss_ticket_number#86, ss_item_sk#85] -Right keys [2]: [sr_ticket_number#98, sr_item_sk#97] +Left keys [2]: [ss_ticket_number#88, ss_item_sk#87] +Right keys [2]: [sr_ticket_number#100, sr_item_sk#99] Join type: LeftOuter Join condition: None (101) Project [codegen id : 39] -Output [7]: [d_year#96, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, (ss_quantity#87 - coalesce(sr_return_quantity#99, 0)) AS sales_cnt#39, (ss_ext_sales_price#88 - coalesce(sr_return_amt#100, 0.00)) AS sales_amt#40] -Input [13]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96, sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100] +Output [7]: [d_year#98, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, (ss_quantity#89 - coalesce(sr_return_quantity#101, 0)) AS sales_cnt#103, (ss_ext_sales_price#90 - coalesce(sr_return_amt#102, 0.00)) AS sales_amt#104] +Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98, sr_item_sk#99, sr_ticket_number#100, sr_return_quantity#101, sr_return_amt#102] (102) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] +Output [5]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#105), dynamicpruningexpression(ws_sold_date_sk#105 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ws_sold_date_sk#109), dynamicpruningexpression(ws_sold_date_sk#109 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (103) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] +Input [5]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109] (104) Filter [codegen id : 42] -Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] -Condition : isnotnull(ws_item_sk#101) +Input [5]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109] +Condition : isnotnull(ws_item_sk#105) (105) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] (106) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#101] -Right keys [1]: [i_item_sk#106] +Left keys [1]: [ws_item_sk#105] +Right keys [1]: [i_item_sk#110] Join type: Inner Join condition: None (107) Project [codegen id : 42] -Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Input [10]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Output [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Input [10]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] (108) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#111, d_year#112] +Output [2]: [d_date_sk#115, d_year#116] (109) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#105] -Right keys [1]: [d_date_sk#111] +Left keys [1]: [ws_sold_date_sk#109] +Right keys [1]: [d_date_sk#115] Join type: Inner Join condition: None (110) Project [codegen id : 42] -Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Input [11]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] +Output [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Input [11]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] (111) Exchange -Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#102, ws_item_sk#101, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Input [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: hashpartitioning(ws_order_number#106, ws_item_sk#105, 5), ENSURE_REQUIREMENTS, [plan_id=13] (112) Sort [codegen id : 43] -Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_order_number#102 ASC NULLS FIRST, ws_item_sk#101 ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: [ws_order_number#106 ASC NULLS FIRST, ws_item_sk#105 ASC NULLS FIRST], false, 0 (113) ReusedExchange [Reuses operator id: 58] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] (114) Sort [codegen id : 45] -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST], false, 0 (115) SortMergeJoin [codegen id : 46] -Left keys [2]: [ws_order_number#102, ws_item_sk#101] -Right keys [2]: [wr_order_number#114, wr_item_sk#113] +Left keys [2]: [ws_order_number#106, ws_item_sk#105] +Right keys [2]: [wr_order_number#118, wr_item_sk#117] Join type: LeftOuter Join condition: None (116) Project [codegen id : 46] -Output [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#103 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#58, (ws_ext_sales_price#104 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#59] -Input [13]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Output [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#107 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#121, (ws_ext_sales_price#108 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#122] +Input [13]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] (117) Union (118) HashAggregate [codegen id : 47] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] (119) Exchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86, 5), ENSURE_REQUIREMENTS, [plan_id=14] (120) HashAggregate [codegen id : 48] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] (121) HashAggregate [codegen id : 48] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum#60, sum#117] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118] +Functions [2]: [partial_sum(sales_cnt#85), partial_sum(UnscaledValue(sales_amt#86))] +Aggregate Attributes [2]: [sum#123, sum#124] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] (122) Exchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, [plan_id=15] (123) HashAggregate [codegen id : 49] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#64, sum(UnscaledValue(sales_amt#21))#65] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum(sales_cnt#20)#64 AS sales_cnt#119, MakeDecimal(sum(UnscaledValue(sales_amt#21))#65,18,2) AS sales_amt#120] +Functions [2]: [sum(sales_cnt#85), sum(UnscaledValue(sales_amt#86))] +Aggregate Attributes [2]: [sum(sales_cnt#85)#64, sum(UnscaledValue(sales_amt#86))#65] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum(sales_cnt#85)#64 AS sales_cnt#127, MakeDecimal(sum(UnscaledValue(sales_amt#86))#65,18,2) AS sales_amt#128] (124) Filter [codegen id : 49] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] -Condition : isnotnull(sales_cnt#119) +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] +Condition : isnotnull(sales_cnt#127) (125) Exchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, [plan_id=16] (126) Sort [codegen id : 50] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] Arguments: [i_brand_id#75 ASC NULLS FIRST, i_class_id#76 ASC NULLS FIRST, i_category_id#77 ASC NULLS FIRST, i_manufact_id#78 ASC NULLS FIRST], false, 0 (127) SortMergeJoin [codegen id : 51] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Right keys [4]: [i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] Join type: Inner -Join condition: ((cast(sales_cnt#66 as decimal(17,2)) / cast(sales_cnt#119 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#66 as decimal(17,2)) / cast(sales_cnt#127 as decimal(17,2))) < 0.90000000000000000000) (128) Project [codegen id : 51] -Output [10]: [d_year#80 AS prev_year#121, d_year#14 AS year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#119 AS prev_yr_cnt#123, sales_cnt#66 AS curr_yr_cnt#124, (sales_cnt#66 - sales_cnt#119) AS sales_cnt_diff#125, (sales_amt#67 - sales_amt#120) AS sales_amt_diff#126] -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67, d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] +Output [10]: [d_year#80 AS prev_year#129, d_year#14 AS year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#127 AS prev_yr_cnt#131, sales_cnt#66 AS curr_yr_cnt#132, (sales_cnt#66 - sales_cnt#127) AS sales_cnt_diff#133, (sales_amt#67 - sales_amt#128) AS sales_amt_diff#134] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67, d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] (129) TakeOrderedAndProject -Input [10]: [prev_year#121, year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#123, curr_yr_cnt#124, sales_cnt_diff#125, sales_amt_diff#126] -Arguments: 100, [sales_cnt_diff#125 ASC NULLS FIRST, sales_amt_diff#126 ASC NULLS FIRST], [prev_year#121, year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#123, curr_yr_cnt#124, sales_cnt_diff#125, sales_amt_diff#126] +Input [10]: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] +Arguments: 100, [sales_cnt_diff#133 ASC NULLS FIRST, sales_amt_diff#134 ASC NULLS FIRST], [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] ===== Subqueries ===== @@ -772,8 +772,8 @@ Condition : ((isnotnull(d_year#80) AND (d_year#80 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#79, d_year#80] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#73 +Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#73 -Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#105 IN dynamicpruning#73 +Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#109 IN dynamicpruning#73 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt index 65f1a88c277b..8fcd79ff1552 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt @@ -517,204 +517,204 @@ Join type: LeftOuter Join condition: None (86) Project [codegen id : 32] -Output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#20, (cs_ext_sales_price#71 - coalesce(cr_return_amount#84, 0.00)) AS sales_amt#21] +Output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#85, (cs_ext_sales_price#71 - coalesce(cr_return_amount#84, 0.00)) AS sales_amt#86] Input [13]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80, cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] (87) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#89), dynamicpruningexpression(ss_sold_date_sk#89 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (88) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] (89) Filter [codegen id : 35] -Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Condition : isnotnull(ss_item_sk#85) +Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +Condition : isnotnull(ss_item_sk#87) (90) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94] +Output [5]: [i_item_sk#92, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96] (91) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#85] -Right keys [1]: [i_item_sk#90] +Left keys [1]: [ss_item_sk#87] +Right keys [1]: [i_item_sk#92] Join type: Inner Join condition: None (92) Project [codegen id : 35] -Output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94] -Input [10]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94] +Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96] +Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#92, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96] (93) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#95, d_year#96] +Output [2]: [d_date_sk#97, d_year#98] (94) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#89] -Right keys [1]: [d_date_sk#95] +Left keys [1]: [ss_sold_date_sk#91] +Right keys [1]: [d_date_sk#97] Join type: Inner Join condition: None (95) Project [codegen id : 35] -Output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96] -Input [11]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_date_sk#95, d_year#96] +Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98] +Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_date_sk#97, d_year#98] (96) Exchange -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96] -Arguments: hashpartitioning(ss_ticket_number#86, ss_item_sk#85, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98] +Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=12] (97) Sort [codegen id : 36] -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96] -Arguments: [ss_ticket_number#86 ASC NULLS FIRST, ss_item_sk#85 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98] +Arguments: [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST], false, 0 (98) ReusedExchange [Reuses operator id: 39] -Output [4]: [sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100] +Output [4]: [sr_item_sk#99, sr_ticket_number#100, sr_return_quantity#101, sr_return_amt#102] (99) Sort [codegen id : 38] -Input [4]: [sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100] -Arguments: [sr_ticket_number#98 ASC NULLS FIRST, sr_item_sk#97 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#99, sr_ticket_number#100, sr_return_quantity#101, sr_return_amt#102] +Arguments: [sr_ticket_number#100 ASC NULLS FIRST, sr_item_sk#99 ASC NULLS FIRST], false, 0 (100) SortMergeJoin [codegen id : 39] -Left keys [2]: [ss_ticket_number#86, ss_item_sk#85] -Right keys [2]: [sr_ticket_number#98, sr_item_sk#97] +Left keys [2]: [ss_ticket_number#88, ss_item_sk#87] +Right keys [2]: [sr_ticket_number#100, sr_item_sk#99] Join type: LeftOuter Join condition: None (101) Project [codegen id : 39] -Output [7]: [d_year#96, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, (ss_quantity#87 - coalesce(sr_return_quantity#99, 0)) AS sales_cnt#39, (ss_ext_sales_price#88 - coalesce(sr_return_amt#100, 0.00)) AS sales_amt#40] -Input [13]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96, sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100] +Output [7]: [d_year#98, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, (ss_quantity#89 - coalesce(sr_return_quantity#101, 0)) AS sales_cnt#103, (ss_ext_sales_price#90 - coalesce(sr_return_amt#102, 0.00)) AS sales_amt#104] +Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#93, i_class_id#94, i_category_id#95, i_manufact_id#96, d_year#98, sr_item_sk#99, sr_ticket_number#100, sr_return_quantity#101, sr_return_amt#102] (102) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] +Output [5]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#105), dynamicpruningexpression(ws_sold_date_sk#105 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ws_sold_date_sk#109), dynamicpruningexpression(ws_sold_date_sk#109 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (103) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] +Input [5]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109] (104) Filter [codegen id : 42] -Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] -Condition : isnotnull(ws_item_sk#101) +Input [5]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109] +Condition : isnotnull(ws_item_sk#105) (105) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] (106) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#101] -Right keys [1]: [i_item_sk#106] +Left keys [1]: [ws_item_sk#105] +Right keys [1]: [i_item_sk#110] Join type: Inner Join condition: None (107) Project [codegen id : 42] -Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Input [10]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Output [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Input [10]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] (108) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#111, d_year#112] +Output [2]: [d_date_sk#115, d_year#116] (109) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#105] -Right keys [1]: [d_date_sk#111] +Left keys [1]: [ws_sold_date_sk#109] +Right keys [1]: [d_date_sk#115] Join type: Inner Join condition: None (110) Project [codegen id : 42] -Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Input [11]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] +Output [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Input [11]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, ws_sold_date_sk#109, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] (111) Exchange -Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#102, ws_item_sk#101, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Input [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: hashpartitioning(ws_order_number#106, ws_item_sk#105, 5), ENSURE_REQUIREMENTS, [plan_id=13] (112) Sort [codegen id : 43] -Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_order_number#102 ASC NULLS FIRST, ws_item_sk#101 ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: [ws_order_number#106 ASC NULLS FIRST, ws_item_sk#105 ASC NULLS FIRST], false, 0 (113) ReusedExchange [Reuses operator id: 58] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] (114) Sort [codegen id : 45] -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST], false, 0 (115) SortMergeJoin [codegen id : 46] -Left keys [2]: [ws_order_number#102, ws_item_sk#101] -Right keys [2]: [wr_order_number#114, wr_item_sk#113] +Left keys [2]: [ws_order_number#106, ws_item_sk#105] +Right keys [2]: [wr_order_number#118, wr_item_sk#117] Join type: LeftOuter Join condition: None (116) Project [codegen id : 46] -Output [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#103 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#58, (ws_ext_sales_price#104 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#59] -Input [13]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Output [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#107 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#121, (ws_ext_sales_price#108 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#122] +Input [13]: [ws_item_sk#105, ws_order_number#106, ws_quantity#107, ws_ext_sales_price#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] (117) Union (118) HashAggregate [codegen id : 47] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] (119) Exchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86, 5), ENSURE_REQUIREMENTS, [plan_id=14] (120) HashAggregate [codegen id : 48] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] (121) HashAggregate [codegen id : 48] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum#60, sum#117] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118] +Functions [2]: [partial_sum(sales_cnt#85), partial_sum(UnscaledValue(sales_amt#86))] +Aggregate Attributes [2]: [sum#123, sum#124] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] (122) Exchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, [plan_id=15] (123) HashAggregate [codegen id : 49] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#64, sum(UnscaledValue(sales_amt#21))#65] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum(sales_cnt#20)#64 AS sales_cnt#119, MakeDecimal(sum(UnscaledValue(sales_amt#21))#65,18,2) AS sales_amt#120] +Functions [2]: [sum(sales_cnt#85), sum(UnscaledValue(sales_amt#86))] +Aggregate Attributes [2]: [sum(sales_cnt#85)#64, sum(UnscaledValue(sales_amt#86))#65] +Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum(sales_cnt#85)#64 AS sales_cnt#127, MakeDecimal(sum(UnscaledValue(sales_amt#86))#65,18,2) AS sales_amt#128] (124) Filter [codegen id : 49] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] -Condition : isnotnull(sales_cnt#119) +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] +Condition : isnotnull(sales_cnt#127) (125) Exchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, [plan_id=16] (126) Sort [codegen id : 50] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] +Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] Arguments: [i_brand_id#75 ASC NULLS FIRST, i_class_id#76 ASC NULLS FIRST, i_category_id#77 ASC NULLS FIRST, i_manufact_id#78 ASC NULLS FIRST], false, 0 (127) SortMergeJoin [codegen id : 51] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Right keys [4]: [i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] Join type: Inner -Join condition: ((cast(sales_cnt#66 as decimal(17,2)) / cast(sales_cnt#119 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#66 as decimal(17,2)) / cast(sales_cnt#127 as decimal(17,2))) < 0.90000000000000000000) (128) Project [codegen id : 51] -Output [10]: [d_year#80 AS prev_year#121, d_year#14 AS year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#119 AS prev_yr_cnt#123, sales_cnt#66 AS curr_yr_cnt#124, (sales_cnt#66 - sales_cnt#119) AS sales_cnt_diff#125, (sales_amt#67 - sales_amt#120) AS sales_amt_diff#126] -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67, d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120] +Output [10]: [d_year#80 AS prev_year#129, d_year#14 AS year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#127 AS prev_yr_cnt#131, sales_cnt#66 AS curr_yr_cnt#132, (sales_cnt#66 - sales_cnt#127) AS sales_cnt_diff#133, (sales_amt#67 - sales_amt#128) AS sales_amt_diff#134] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67, d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] (129) TakeOrderedAndProject -Input [10]: [prev_year#121, year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#123, curr_yr_cnt#124, sales_cnt_diff#125, sales_amt_diff#126] -Arguments: 100, [sales_cnt_diff#125 ASC NULLS FIRST, sales_amt_diff#126 ASC NULLS FIRST], [prev_year#121, year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#123, curr_yr_cnt#124, sales_cnt_diff#125, sales_amt_diff#126] +Input [10]: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] +Arguments: 100, [sales_cnt_diff#133 ASC NULLS FIRST, sales_amt_diff#134 ASC NULLS FIRST], [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] ===== Subqueries ===== @@ -772,8 +772,8 @@ Condition : ((isnotnull(d_year#80) AND (d_year#80 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#79, d_year#80] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#73 +Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#73 -Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#105 IN dynamicpruning#73 +Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#109 IN dynamicpruning#73 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt index 655581148e48..ff375a178e42 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt @@ -508,60 +508,60 @@ Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33 Results [5]: [channel#30, id#31, cast(sum(sales#14)#106 as decimal(37,2)) AS sales#109, cast(sum(returns#32)#107 as decimal(37,2)) AS returns#110, cast(sum(profit#33)#108 as decimal(38,2)) AS profit#111] (84) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Output [8]: [channel#112, id#113, sum#100, isEmpty#101, sum#114, isEmpty#115, sum#116, isEmpty#117] (85) HashAggregate [codegen id : 48] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [2]: [channel#30, id#31] -Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] -Results [4]: [channel#30, sum(sales#14)#106 AS sales#112, sum(returns#32)#107 AS returns#113, sum(profit#33)#108 AS profit#114] +Input [8]: [channel#112, id#113, sum#100, isEmpty#101, sum#114, isEmpty#115, sum#116, isEmpty#117] +Keys [2]: [channel#112, id#113] +Functions [3]: [sum(sales#14), sum(returns#118), sum(profit#119)] +Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#118)#107, sum(profit#119)#108] +Results [4]: [channel#112, sum(sales#14)#106 AS sales#120, sum(returns#118)#107 AS returns#121, sum(profit#119)#108 AS profit#122] (86) HashAggregate [codegen id : 48] -Input [4]: [channel#30, sales#112, returns#113, profit#114] -Keys [1]: [channel#30] -Functions [3]: [partial_sum(sales#112), partial_sum(returns#113), partial_sum(profit#114)] -Aggregate Attributes [6]: [sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] -Results [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] +Input [4]: [channel#112, sales#120, returns#121, profit#122] +Keys [1]: [channel#112] +Functions [3]: [partial_sum(sales#120), partial_sum(returns#121), partial_sum(profit#122)] +Aggregate Attributes [6]: [sum#123, isEmpty#124, sum#125, isEmpty#126, sum#127, isEmpty#128] +Results [7]: [channel#112, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] (87) Exchange -Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Arguments: hashpartitioning(channel#30, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Input [7]: [channel#112, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Arguments: hashpartitioning(channel#112, 5), ENSURE_REQUIREMENTS, [plan_id=13] (88) HashAggregate [codegen id : 49] -Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Keys [1]: [channel#30] -Functions [3]: [sum(sales#112), sum(returns#113), sum(profit#114)] -Aggregate Attributes [3]: [sum(sales#112)#127, sum(returns#113)#128, sum(profit#114)#129] -Results [5]: [channel#30, null AS id#130, sum(sales#112)#127 AS sales#131, sum(returns#113)#128 AS returns#132, sum(profit#114)#129 AS profit#133] +Input [7]: [channel#112, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Keys [1]: [channel#112] +Functions [3]: [sum(sales#120), sum(returns#121), sum(profit#122)] +Aggregate Attributes [3]: [sum(sales#120)#135, sum(returns#121)#136, sum(profit#122)#137] +Results [5]: [channel#112, null AS id#138, sum(sales#120)#135 AS sales#139, sum(returns#121)#136 AS returns#140, sum(profit#122)#137 AS profit#141] (89) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Output [8]: [channel#142, id#143, sum#100, isEmpty#101, sum#144, isEmpty#145, sum#146, isEmpty#147] (90) HashAggregate [codegen id : 73] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [2]: [channel#30, id#31] -Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] -Results [3]: [sum(sales#14)#106 AS sales#134, sum(returns#32)#107 AS returns#135, sum(profit#33)#108 AS profit#136] +Input [8]: [channel#142, id#143, sum#100, isEmpty#101, sum#144, isEmpty#145, sum#146, isEmpty#147] +Keys [2]: [channel#142, id#143] +Functions [3]: [sum(sales#14), sum(returns#148), sum(profit#149)] +Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#148)#107, sum(profit#149)#108] +Results [3]: [sum(sales#14)#106 AS sales#150, sum(returns#148)#107 AS returns#151, sum(profit#149)#108 AS profit#152] (91) HashAggregate [codegen id : 73] -Input [3]: [sales#134, returns#135, profit#136] +Input [3]: [sales#150, returns#151, profit#152] Keys: [] -Functions [3]: [partial_sum(sales#134), partial_sum(returns#135), partial_sum(profit#136)] -Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] -Results [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Functions [3]: [partial_sum(sales#150), partial_sum(returns#151), partial_sum(profit#152)] +Aggregate Attributes [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Results [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] (92) Exchange -Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Input [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] (93) HashAggregate [codegen id : 74] -Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Input [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] Keys: [] -Functions [3]: [sum(sales#134), sum(returns#135), sum(profit#136)] -Aggregate Attributes [3]: [sum(sales#134)#149, sum(returns#135)#150, sum(profit#136)#151] -Results [5]: [null AS channel#152, null AS id#153, sum(sales#134)#149 AS sales#154, sum(returns#135)#150 AS returns#155, sum(profit#136)#151 AS profit#156] +Functions [3]: [sum(sales#150), sum(returns#151), sum(profit#152)] +Aggregate Attributes [3]: [sum(sales#150)#165, sum(returns#151)#166, sum(profit#152)#167] +Results [5]: [null AS channel#168, null AS id#169, sum(sales#150)#165 AS sales#170, sum(returns#151)#166 AS returns#171, sum(profit#152)#167 AS profit#172] (94) Union @@ -598,22 +598,22 @@ BroadcastExchange (103) (99) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#157] +Output [2]: [d_date_sk#6, d_date#173] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_date#157] +Input [2]: [d_date_sk#6, d_date#173] (101) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_date#157] -Condition : (((isnotnull(d_date#157) AND (d_date#157 >= 1998-08-04)) AND (d_date#157 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_date#173] +Condition : (((isnotnull(d_date#173) AND (d_date#173 >= 1998-08-04)) AND (d_date#173 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) (102) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_date#157] +Input [2]: [d_date_sk#6, d_date#173] (103) BroadcastExchange Input [1]: [d_date_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt index 908ee7831ea0..77463b94bcc0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt @@ -508,60 +508,60 @@ Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33 Results [5]: [channel#30, id#31, cast(sum(sales#14)#106 as decimal(37,2)) AS sales#109, cast(sum(returns#32)#107 as decimal(37,2)) AS returns#110, cast(sum(profit#33)#108 as decimal(38,2)) AS profit#111] (84) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Output [8]: [channel#112, id#113, sum#100, isEmpty#101, sum#114, isEmpty#115, sum#116, isEmpty#117] (85) HashAggregate [codegen id : 48] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [2]: [channel#30, id#31] -Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] -Results [4]: [channel#30, sum(sales#14)#106 AS sales#112, sum(returns#32)#107 AS returns#113, sum(profit#33)#108 AS profit#114] +Input [8]: [channel#112, id#113, sum#100, isEmpty#101, sum#114, isEmpty#115, sum#116, isEmpty#117] +Keys [2]: [channel#112, id#113] +Functions [3]: [sum(sales#14), sum(returns#118), sum(profit#119)] +Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#118)#107, sum(profit#119)#108] +Results [4]: [channel#112, sum(sales#14)#106 AS sales#120, sum(returns#118)#107 AS returns#121, sum(profit#119)#108 AS profit#122] (86) HashAggregate [codegen id : 48] -Input [4]: [channel#30, sales#112, returns#113, profit#114] -Keys [1]: [channel#30] -Functions [3]: [partial_sum(sales#112), partial_sum(returns#113), partial_sum(profit#114)] -Aggregate Attributes [6]: [sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] -Results [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] +Input [4]: [channel#112, sales#120, returns#121, profit#122] +Keys [1]: [channel#112] +Functions [3]: [partial_sum(sales#120), partial_sum(returns#121), partial_sum(profit#122)] +Aggregate Attributes [6]: [sum#123, isEmpty#124, sum#125, isEmpty#126, sum#127, isEmpty#128] +Results [7]: [channel#112, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] (87) Exchange -Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Arguments: hashpartitioning(channel#30, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Input [7]: [channel#112, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Arguments: hashpartitioning(channel#112, 5), ENSURE_REQUIREMENTS, [plan_id=13] (88) HashAggregate [codegen id : 49] -Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Keys [1]: [channel#30] -Functions [3]: [sum(sales#112), sum(returns#113), sum(profit#114)] -Aggregate Attributes [3]: [sum(sales#112)#127, sum(returns#113)#128, sum(profit#114)#129] -Results [5]: [channel#30, null AS id#130, sum(sales#112)#127 AS sales#131, sum(returns#113)#128 AS returns#132, sum(profit#114)#129 AS profit#133] +Input [7]: [channel#112, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Keys [1]: [channel#112] +Functions [3]: [sum(sales#120), sum(returns#121), sum(profit#122)] +Aggregate Attributes [3]: [sum(sales#120)#135, sum(returns#121)#136, sum(profit#122)#137] +Results [5]: [channel#112, null AS id#138, sum(sales#120)#135 AS sales#139, sum(returns#121)#136 AS returns#140, sum(profit#122)#137 AS profit#141] (89) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Output [8]: [channel#142, id#143, sum#100, isEmpty#101, sum#144, isEmpty#145, sum#146, isEmpty#147] (90) HashAggregate [codegen id : 73] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [2]: [channel#30, id#31] -Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] -Results [3]: [sum(sales#14)#106 AS sales#134, sum(returns#32)#107 AS returns#135, sum(profit#33)#108 AS profit#136] +Input [8]: [channel#142, id#143, sum#100, isEmpty#101, sum#144, isEmpty#145, sum#146, isEmpty#147] +Keys [2]: [channel#142, id#143] +Functions [3]: [sum(sales#14), sum(returns#148), sum(profit#149)] +Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#148)#107, sum(profit#149)#108] +Results [3]: [sum(sales#14)#106 AS sales#150, sum(returns#148)#107 AS returns#151, sum(profit#149)#108 AS profit#152] (91) HashAggregate [codegen id : 73] -Input [3]: [sales#134, returns#135, profit#136] +Input [3]: [sales#150, returns#151, profit#152] Keys: [] -Functions [3]: [partial_sum(sales#134), partial_sum(returns#135), partial_sum(profit#136)] -Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] -Results [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Functions [3]: [partial_sum(sales#150), partial_sum(returns#151), partial_sum(profit#152)] +Aggregate Attributes [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Results [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] (92) Exchange -Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Input [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] (93) HashAggregate [codegen id : 74] -Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Input [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] Keys: [] -Functions [3]: [sum(sales#134), sum(returns#135), sum(profit#136)] -Aggregate Attributes [3]: [sum(sales#134)#149, sum(returns#135)#150, sum(profit#136)#151] -Results [5]: [null AS channel#152, null AS id#153, sum(sales#134)#149 AS sales#154, sum(returns#135)#150 AS returns#155, sum(profit#136)#151 AS profit#156] +Functions [3]: [sum(sales#150), sum(returns#151), sum(profit#152)] +Aggregate Attributes [3]: [sum(sales#150)#165, sum(returns#151)#166, sum(profit#152)#167] +Results [5]: [null AS channel#168, null AS id#169, sum(sales#150)#165 AS sales#170, sum(returns#151)#166 AS returns#171, sum(profit#152)#167 AS profit#172] (94) Union @@ -598,22 +598,22 @@ BroadcastExchange (103) (99) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#157] +Output [2]: [d_date_sk#6, d_date#173] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_date#157] +Input [2]: [d_date_sk#6, d_date#173] (101) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_date#157] -Condition : (((isnotnull(d_date#157) AND (d_date#157 >= 1998-08-04)) AND (d_date#157 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_date#173] +Condition : (((isnotnull(d_date#173) AND (d_date#173 >= 1998-08-04)) AND (d_date#173 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) (102) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_date#157] +Input [2]: [d_date_sk#6, d_date#173] (103) BroadcastExchange Input [1]: [d_date_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt index edb7e3f3de14..f41522bbc37c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt @@ -701,433 +701,433 @@ Input [6]: [s_store_id#131, sum#132, sum#133, isEmpty#134, sum#135, isEmpty#136] Keys [1]: [s_store_id#131] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#137)), sum(coalesce(cast(sr_return_amt#138 as decimal(12,2)), 0.00)), sum((ss_net_profit#139 - coalesce(cast(sr_net_loss#140 as decimal(12,2)), 0.00)))] Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#137))#35, sum(coalesce(cast(sr_return_amt#138 as decimal(12,2)), 0.00))#36, sum((ss_net_profit#139 - coalesce(cast(sr_net_loss#140 as decimal(12,2)), 0.00)))#37] -Results [5]: [store channel AS channel#38, concat(store, s_store_id#131) AS id#39, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#137))#35,17,2) AS sales#40, sum(coalesce(cast(sr_return_amt#138 as decimal(12,2)), 0.00))#36 AS returns#41, sum((ss_net_profit#139 - coalesce(cast(sr_net_loss#140 as decimal(12,2)), 0.00)))#37 AS profit#42] +Results [5]: [store channel AS channel#141, concat(store, s_store_id#131) AS id#142, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#137))#35,17,2) AS sales#40, sum(coalesce(cast(sr_return_amt#138 as decimal(12,2)), 0.00))#36 AS returns#41, sum((ss_net_profit#139 - coalesce(cast(sr_net_loss#140 as decimal(12,2)), 0.00)))#37 AS profit#42] (108) ReusedExchange [Reuses operator id: 69] -Output [6]: [cp_catalog_page_id#141, sum#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Output [6]: [cp_catalog_page_id#143, sum#144, sum#145, isEmpty#146, sum#147, isEmpty#148] (109) HashAggregate [codegen id : 52] -Input [6]: [cp_catalog_page_id#141, sum#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Keys [1]: [cp_catalog_page_id#141] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#147)), sum(coalesce(cast(cr_return_amount#148 as decimal(12,2)), 0.00)), sum((cs_net_profit#149 - coalesce(cast(cr_net_loss#150 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#147))#70, sum(coalesce(cast(cr_return_amount#148 as decimal(12,2)), 0.00))#71, sum((cs_net_profit#149 - coalesce(cast(cr_net_loss#150 as decimal(12,2)), 0.00)))#72] -Results [5]: [catalog channel AS channel#73, concat(catalog_page, cp_catalog_page_id#141) AS id#74, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#147))#70,17,2) AS sales#75, sum(coalesce(cast(cr_return_amount#148 as decimal(12,2)), 0.00))#71 AS returns#76, sum((cs_net_profit#149 - coalesce(cast(cr_net_loss#150 as decimal(12,2)), 0.00)))#72 AS profit#77] +Input [6]: [cp_catalog_page_id#143, sum#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Keys [1]: [cp_catalog_page_id#143] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#149)), sum(coalesce(cast(cr_return_amount#150 as decimal(12,2)), 0.00)), sum((cs_net_profit#151 - coalesce(cast(cr_net_loss#152 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#149))#70, sum(coalesce(cast(cr_return_amount#150 as decimal(12,2)), 0.00))#71, sum((cs_net_profit#151 - coalesce(cast(cr_net_loss#152 as decimal(12,2)), 0.00)))#72] +Results [5]: [catalog channel AS channel#153, concat(catalog_page, cp_catalog_page_id#143) AS id#154, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#149))#70,17,2) AS sales#75, sum(coalesce(cast(cr_return_amount#150 as decimal(12,2)), 0.00))#71 AS returns#76, sum((cs_net_profit#151 - coalesce(cast(cr_net_loss#152 as decimal(12,2)), 0.00)))#72 AS profit#77] (110) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#151, ws_web_site_sk#152, ws_promo_sk#153, ws_order_number#154, ws_ext_sales_price#155, ws_net_profit#156, ws_sold_date_sk#157] +Output [7]: [ws_item_sk#155, ws_web_site_sk#156, ws_promo_sk#157, ws_order_number#158, ws_ext_sales_price#159, ws_net_profit#160, ws_sold_date_sk#161] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#157), dynamicpruningexpression(ws_sold_date_sk#157 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#161), dynamicpruningexpression(ws_sold_date_sk#161 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (111) ColumnarToRow [codegen id : 53] -Input [7]: [ws_item_sk#151, ws_web_site_sk#152, ws_promo_sk#153, ws_order_number#154, ws_ext_sales_price#155, ws_net_profit#156, ws_sold_date_sk#157] +Input [7]: [ws_item_sk#155, ws_web_site_sk#156, ws_promo_sk#157, ws_order_number#158, ws_ext_sales_price#159, ws_net_profit#160, ws_sold_date_sk#161] (112) Filter [codegen id : 53] -Input [7]: [ws_item_sk#151, ws_web_site_sk#152, ws_promo_sk#153, ws_order_number#154, ws_ext_sales_price#155, ws_net_profit#156, ws_sold_date_sk#157] -Condition : ((isnotnull(ws_web_site_sk#152) AND isnotnull(ws_item_sk#151)) AND isnotnull(ws_promo_sk#153)) +Input [7]: [ws_item_sk#155, ws_web_site_sk#156, ws_promo_sk#157, ws_order_number#158, ws_ext_sales_price#159, ws_net_profit#160, ws_sold_date_sk#161] +Condition : ((isnotnull(ws_web_site_sk#156) AND isnotnull(ws_item_sk#155)) AND isnotnull(ws_promo_sk#157)) (113) Exchange -Input [7]: [ws_item_sk#151, ws_web_site_sk#152, ws_promo_sk#153, ws_order_number#154, ws_ext_sales_price#155, ws_net_profit#156, ws_sold_date_sk#157] -Arguments: hashpartitioning(ws_item_sk#151, ws_order_number#154, 5), ENSURE_REQUIREMENTS, [plan_id=16] +Input [7]: [ws_item_sk#155, ws_web_site_sk#156, ws_promo_sk#157, ws_order_number#158, ws_ext_sales_price#159, ws_net_profit#160, ws_sold_date_sk#161] +Arguments: hashpartitioning(ws_item_sk#155, ws_order_number#158, 5), ENSURE_REQUIREMENTS, [plan_id=16] (114) Sort [codegen id : 54] -Input [7]: [ws_item_sk#151, ws_web_site_sk#152, ws_promo_sk#153, ws_order_number#154, ws_ext_sales_price#155, ws_net_profit#156, ws_sold_date_sk#157] -Arguments: [ws_item_sk#151 ASC NULLS FIRST, ws_order_number#154 ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#155, ws_web_site_sk#156, ws_promo_sk#157, ws_order_number#158, ws_ext_sales_price#159, ws_net_profit#160, ws_sold_date_sk#161] +Arguments: [ws_item_sk#155 ASC NULLS FIRST, ws_order_number#158 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: 80] -Output [4]: [wr_item_sk#158, wr_order_number#159, wr_return_amt#160, wr_net_loss#161] +Output [4]: [wr_item_sk#162, wr_order_number#163, wr_return_amt#164, wr_net_loss#165] (116) Sort [codegen id : 56] -Input [4]: [wr_item_sk#158, wr_order_number#159, wr_return_amt#160, wr_net_loss#161] -Arguments: [wr_item_sk#158 ASC NULLS FIRST, wr_order_number#159 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#162, wr_order_number#163, wr_return_amt#164, wr_net_loss#165] +Arguments: [wr_item_sk#162 ASC NULLS FIRST, wr_order_number#163 ASC NULLS FIRST], false, 0 (117) SortMergeJoin [codegen id : 61] -Left keys [2]: [ws_item_sk#151, ws_order_number#154] -Right keys [2]: [wr_item_sk#158, wr_order_number#159] +Left keys [2]: [ws_item_sk#155, ws_order_number#158] +Right keys [2]: [wr_item_sk#162, wr_order_number#163] Join type: LeftOuter Join condition: None (118) Project [codegen id : 61] -Output [8]: [ws_item_sk#151, ws_web_site_sk#152, ws_promo_sk#153, ws_ext_sales_price#155, ws_net_profit#156, ws_sold_date_sk#157, wr_return_amt#160, wr_net_loss#161] -Input [11]: [ws_item_sk#151, ws_web_site_sk#152, ws_promo_sk#153, ws_order_number#154, ws_ext_sales_price#155, ws_net_profit#156, ws_sold_date_sk#157, wr_item_sk#158, wr_order_number#159, wr_return_amt#160, wr_net_loss#161] +Output [8]: [ws_item_sk#155, ws_web_site_sk#156, ws_promo_sk#157, ws_ext_sales_price#159, ws_net_profit#160, ws_sold_date_sk#161, wr_return_amt#164, wr_net_loss#165] +Input [11]: [ws_item_sk#155, ws_web_site_sk#156, ws_promo_sk#157, ws_order_number#158, ws_ext_sales_price#159, ws_net_profit#160, ws_sold_date_sk#161, wr_item_sk#162, wr_order_number#163, wr_return_amt#164, wr_net_loss#165] (119) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#162] +Output [1]: [i_item_sk#166] (120) BroadcastHashJoin [codegen id : 61] -Left keys [1]: [ws_item_sk#151] -Right keys [1]: [i_item_sk#162] +Left keys [1]: [ws_item_sk#155] +Right keys [1]: [i_item_sk#166] Join type: Inner Join condition: None (121) Project [codegen id : 61] -Output [7]: [ws_web_site_sk#152, ws_promo_sk#153, ws_ext_sales_price#155, ws_net_profit#156, ws_sold_date_sk#157, wr_return_amt#160, wr_net_loss#161] -Input [9]: [ws_item_sk#151, ws_web_site_sk#152, ws_promo_sk#153, ws_ext_sales_price#155, ws_net_profit#156, ws_sold_date_sk#157, wr_return_amt#160, wr_net_loss#161, i_item_sk#162] +Output [7]: [ws_web_site_sk#156, ws_promo_sk#157, ws_ext_sales_price#159, ws_net_profit#160, ws_sold_date_sk#161, wr_return_amt#164, wr_net_loss#165] +Input [9]: [ws_item_sk#155, ws_web_site_sk#156, ws_promo_sk#157, ws_ext_sales_price#159, ws_net_profit#160, ws_sold_date_sk#161, wr_return_amt#164, wr_net_loss#165, i_item_sk#166] (122) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#163] +Output [1]: [p_promo_sk#167] (123) BroadcastHashJoin [codegen id : 61] -Left keys [1]: [ws_promo_sk#153] -Right keys [1]: [p_promo_sk#163] +Left keys [1]: [ws_promo_sk#157] +Right keys [1]: [p_promo_sk#167] Join type: Inner Join condition: None (124) Project [codegen id : 61] -Output [6]: [ws_web_site_sk#152, ws_ext_sales_price#155, ws_net_profit#156, ws_sold_date_sk#157, wr_return_amt#160, wr_net_loss#161] -Input [8]: [ws_web_site_sk#152, ws_promo_sk#153, ws_ext_sales_price#155, ws_net_profit#156, ws_sold_date_sk#157, wr_return_amt#160, wr_net_loss#161, p_promo_sk#163] +Output [6]: [ws_web_site_sk#156, ws_ext_sales_price#159, ws_net_profit#160, ws_sold_date_sk#161, wr_return_amt#164, wr_net_loss#165] +Input [8]: [ws_web_site_sk#156, ws_promo_sk#157, ws_ext_sales_price#159, ws_net_profit#160, ws_sold_date_sk#161, wr_return_amt#164, wr_net_loss#165, p_promo_sk#167] (125) ReusedExchange [Reuses operator id: 221] -Output [1]: [d_date_sk#164] +Output [1]: [d_date_sk#168] (126) BroadcastHashJoin [codegen id : 61] -Left keys [1]: [ws_sold_date_sk#157] -Right keys [1]: [d_date_sk#164] +Left keys [1]: [ws_sold_date_sk#161] +Right keys [1]: [d_date_sk#168] Join type: Inner Join condition: None (127) Project [codegen id : 61] -Output [5]: [ws_web_site_sk#152, ws_ext_sales_price#155, ws_net_profit#156, wr_return_amt#160, wr_net_loss#161] -Input [7]: [ws_web_site_sk#152, ws_ext_sales_price#155, ws_net_profit#156, ws_sold_date_sk#157, wr_return_amt#160, wr_net_loss#161, d_date_sk#164] +Output [5]: [ws_web_site_sk#156, ws_ext_sales_price#159, ws_net_profit#160, wr_return_amt#164, wr_net_loss#165] +Input [7]: [ws_web_site_sk#156, ws_ext_sales_price#159, ws_net_profit#160, ws_sold_date_sk#161, wr_return_amt#164, wr_net_loss#165, d_date_sk#168] (128) ReusedExchange [Reuses operator id: 96] -Output [2]: [web_site_sk#165, web_site_id#166] +Output [2]: [web_site_sk#169, web_site_id#170] (129) BroadcastHashJoin [codegen id : 61] -Left keys [1]: [ws_web_site_sk#152] -Right keys [1]: [web_site_sk#165] +Left keys [1]: [ws_web_site_sk#156] +Right keys [1]: [web_site_sk#169] Join type: Inner Join condition: None (130) Project [codegen id : 61] -Output [5]: [ws_ext_sales_price#155, ws_net_profit#156, wr_return_amt#160, wr_net_loss#161, web_site_id#166] -Input [7]: [ws_web_site_sk#152, ws_ext_sales_price#155, ws_net_profit#156, wr_return_amt#160, wr_net_loss#161, web_site_sk#165, web_site_id#166] +Output [5]: [ws_ext_sales_price#159, ws_net_profit#160, wr_return_amt#164, wr_net_loss#165, web_site_id#170] +Input [7]: [ws_web_site_sk#156, ws_ext_sales_price#159, ws_net_profit#160, wr_return_amt#164, wr_net_loss#165, web_site_sk#169, web_site_id#170] (131) HashAggregate [codegen id : 61] -Input [5]: [ws_ext_sales_price#155, ws_net_profit#156, wr_return_amt#160, wr_net_loss#161, web_site_id#166] -Keys [1]: [web_site_id#166] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#155)), partial_sum(coalesce(cast(wr_return_amt#160 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#156 - coalesce(cast(wr_net_loss#161 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#167, sum#168, isEmpty#169, sum#170, isEmpty#171] -Results [6]: [web_site_id#166, sum#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Input [5]: [ws_ext_sales_price#159, ws_net_profit#160, wr_return_amt#164, wr_net_loss#165, web_site_id#170] +Keys [1]: [web_site_id#170] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#159)), partial_sum(coalesce(cast(wr_return_amt#164 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#160 - coalesce(cast(wr_net_loss#165 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Results [6]: [web_site_id#170, sum#176, sum#177, isEmpty#178, sum#179, isEmpty#180] (132) Exchange -Input [6]: [web_site_id#166, sum#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: hashpartitioning(web_site_id#166, 5), ENSURE_REQUIREMENTS, [plan_id=17] +Input [6]: [web_site_id#170, sum#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +Arguments: hashpartitioning(web_site_id#170, 5), ENSURE_REQUIREMENTS, [plan_id=17] (133) HashAggregate [codegen id : 62] -Input [6]: [web_site_id#166, sum#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Keys [1]: [web_site_id#166] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#155)), sum(coalesce(cast(wr_return_amt#160 as decimal(12,2)), 0.00)), sum((ws_net_profit#156 - coalesce(cast(wr_net_loss#161 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#155))#105, sum(coalesce(cast(wr_return_amt#160 as decimal(12,2)), 0.00))#106, sum((ws_net_profit#156 - coalesce(cast(wr_net_loss#161 as decimal(12,2)), 0.00)))#107] -Results [5]: [web channel AS channel#108, concat(web_site, web_site_id#166) AS id#109, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#155))#105,17,2) AS sales#110, sum(coalesce(cast(wr_return_amt#160 as decimal(12,2)), 0.00))#106 AS returns#111, sum((ws_net_profit#156 - coalesce(cast(wr_net_loss#161 as decimal(12,2)), 0.00)))#107 AS profit#112] +Input [6]: [web_site_id#170, sum#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +Keys [1]: [web_site_id#170] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#159)), sum(coalesce(cast(wr_return_amt#164 as decimal(12,2)), 0.00)), sum((ws_net_profit#160 - coalesce(cast(wr_net_loss#165 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#159))#105, sum(coalesce(cast(wr_return_amt#164 as decimal(12,2)), 0.00))#106, sum((ws_net_profit#160 - coalesce(cast(wr_net_loss#165 as decimal(12,2)), 0.00)))#107] +Results [5]: [web channel AS channel#181, concat(web_site, web_site_id#170) AS id#182, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#159))#105,17,2) AS sales#110, sum(coalesce(cast(wr_return_amt#164 as decimal(12,2)), 0.00))#106 AS returns#111, sum((ws_net_profit#160 - coalesce(cast(wr_net_loss#165 as decimal(12,2)), 0.00)))#107 AS profit#112] (134) Union (135) HashAggregate [codegen id : 63] -Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] -Keys [2]: [channel#38, id#39] +Input [5]: [channel#141, id#142, sales#40, returns#41, profit#42] +Keys [2]: [channel#141, id#142] Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] Aggregate Attributes [6]: [sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] -Results [8]: [channel#38, id#39, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Results [8]: [channel#141, id#142, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] (136) Exchange -Input [8]: [channel#38, id#39, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, [plan_id=18] +Input [8]: [channel#141, id#142, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Arguments: hashpartitioning(channel#141, id#142, 5), ENSURE_REQUIREMENTS, [plan_id=18] (137) HashAggregate [codegen id : 64] -Input [8]: [channel#38, id#39, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Keys [2]: [channel#38, id#39] +Input [8]: [channel#141, id#142, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Keys [2]: [channel#141, id#142] Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] Aggregate Attributes [3]: [sum(sales#40)#125, sum(returns#41)#126, sum(profit#42)#127] -Results [4]: [channel#38, sum(sales#40)#125 AS sales#177, sum(returns#41)#126 AS returns#178, sum(profit#42)#127 AS profit#179] +Results [4]: [channel#141, sum(sales#40)#125 AS sales#183, sum(returns#41)#126 AS returns#184, sum(profit#42)#127 AS profit#185] (138) HashAggregate [codegen id : 64] -Input [4]: [channel#38, sales#177, returns#178, profit#179] -Keys [1]: [channel#38] -Functions [3]: [partial_sum(sales#177), partial_sum(returns#178), partial_sum(profit#179)] -Aggregate Attributes [6]: [sum#180, isEmpty#181, sum#182, isEmpty#183, sum#184, isEmpty#185] -Results [7]: [channel#38, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] +Input [4]: [channel#141, sales#183, returns#184, profit#185] +Keys [1]: [channel#141] +Functions [3]: [partial_sum(sales#183), partial_sum(returns#184), partial_sum(profit#185)] +Aggregate Attributes [6]: [sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] +Results [7]: [channel#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] (139) Exchange -Input [7]: [channel#38, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] -Arguments: hashpartitioning(channel#38, 5), ENSURE_REQUIREMENTS, [plan_id=19] +Input [7]: [channel#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] +Arguments: hashpartitioning(channel#141, 5), ENSURE_REQUIREMENTS, [plan_id=19] (140) HashAggregate [codegen id : 65] -Input [7]: [channel#38, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] -Keys [1]: [channel#38] -Functions [3]: [sum(sales#177), sum(returns#178), sum(profit#179)] -Aggregate Attributes [3]: [sum(sales#177)#192, sum(returns#178)#193, sum(profit#179)#194] -Results [5]: [channel#38, null AS id#195, sum(sales#177)#192 AS sales#196, sum(returns#178)#193 AS returns#197, sum(profit#179)#194 AS profit#198] +Input [7]: [channel#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] +Keys [1]: [channel#141] +Functions [3]: [sum(sales#183), sum(returns#184), sum(profit#185)] +Aggregate Attributes [3]: [sum(sales#183)#198, sum(returns#184)#199, sum(profit#185)#200] +Results [5]: [channel#141, null AS id#201, sum(sales#183)#198 AS sales#202, sum(returns#184)#199 AS returns#203, sum(profit#185)#200 AS profit#204] (141) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#199, ss_store_sk#200, ss_promo_sk#201, ss_ticket_number#202, ss_ext_sales_price#203, ss_net_profit#204, ss_sold_date_sk#205] +Output [7]: [ss_item_sk#205, ss_store_sk#206, ss_promo_sk#207, ss_ticket_number#208, ss_ext_sales_price#209, ss_net_profit#210, ss_sold_date_sk#211] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#205), dynamicpruningexpression(ss_sold_date_sk#205 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ss_sold_date_sk#211), dynamicpruningexpression(ss_sold_date_sk#211 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] ReadSchema: struct (142) ColumnarToRow [codegen id : 66] -Input [7]: [ss_item_sk#199, ss_store_sk#200, ss_promo_sk#201, ss_ticket_number#202, ss_ext_sales_price#203, ss_net_profit#204, ss_sold_date_sk#205] +Input [7]: [ss_item_sk#205, ss_store_sk#206, ss_promo_sk#207, ss_ticket_number#208, ss_ext_sales_price#209, ss_net_profit#210, ss_sold_date_sk#211] (143) Filter [codegen id : 66] -Input [7]: [ss_item_sk#199, ss_store_sk#200, ss_promo_sk#201, ss_ticket_number#202, ss_ext_sales_price#203, ss_net_profit#204, ss_sold_date_sk#205] -Condition : ((isnotnull(ss_store_sk#200) AND isnotnull(ss_item_sk#199)) AND isnotnull(ss_promo_sk#201)) +Input [7]: [ss_item_sk#205, ss_store_sk#206, ss_promo_sk#207, ss_ticket_number#208, ss_ext_sales_price#209, ss_net_profit#210, ss_sold_date_sk#211] +Condition : ((isnotnull(ss_store_sk#206) AND isnotnull(ss_item_sk#205)) AND isnotnull(ss_promo_sk#207)) (144) Exchange -Input [7]: [ss_item_sk#199, ss_store_sk#200, ss_promo_sk#201, ss_ticket_number#202, ss_ext_sales_price#203, ss_net_profit#204, ss_sold_date_sk#205] -Arguments: hashpartitioning(ss_item_sk#199, ss_ticket_number#202, 5), ENSURE_REQUIREMENTS, [plan_id=20] +Input [7]: [ss_item_sk#205, ss_store_sk#206, ss_promo_sk#207, ss_ticket_number#208, ss_ext_sales_price#209, ss_net_profit#210, ss_sold_date_sk#211] +Arguments: hashpartitioning(ss_item_sk#205, ss_ticket_number#208, 5), ENSURE_REQUIREMENTS, [plan_id=20] (145) Sort [codegen id : 67] -Input [7]: [ss_item_sk#199, ss_store_sk#200, ss_promo_sk#201, ss_ticket_number#202, ss_ext_sales_price#203, ss_net_profit#204, ss_sold_date_sk#205] -Arguments: [ss_item_sk#199 ASC NULLS FIRST, ss_ticket_number#202 ASC NULLS FIRST], false, 0 +Input [7]: [ss_item_sk#205, ss_store_sk#206, ss_promo_sk#207, ss_ticket_number#208, ss_ext_sales_price#209, ss_net_profit#210, ss_sold_date_sk#211] +Arguments: [ss_item_sk#205 ASC NULLS FIRST, ss_ticket_number#208 ASC NULLS FIRST], false, 0 (146) ReusedExchange [Reuses operator id: 10] -Output [4]: [sr_item_sk#206, sr_ticket_number#207, sr_return_amt#208, sr_net_loss#209] +Output [4]: [sr_item_sk#212, sr_ticket_number#213, sr_return_amt#214, sr_net_loss#215] (147) Sort [codegen id : 69] -Input [4]: [sr_item_sk#206, sr_ticket_number#207, sr_return_amt#208, sr_net_loss#209] -Arguments: [sr_item_sk#206 ASC NULLS FIRST, sr_ticket_number#207 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#212, sr_ticket_number#213, sr_return_amt#214, sr_net_loss#215] +Arguments: [sr_item_sk#212 ASC NULLS FIRST, sr_ticket_number#213 ASC NULLS FIRST], false, 0 (148) SortMergeJoin [codegen id : 74] -Left keys [2]: [ss_item_sk#199, ss_ticket_number#202] -Right keys [2]: [sr_item_sk#206, sr_ticket_number#207] +Left keys [2]: [ss_item_sk#205, ss_ticket_number#208] +Right keys [2]: [sr_item_sk#212, sr_ticket_number#213] Join type: LeftOuter Join condition: None (149) Project [codegen id : 74] -Output [8]: [ss_item_sk#199, ss_store_sk#200, ss_promo_sk#201, ss_ext_sales_price#203, ss_net_profit#204, ss_sold_date_sk#205, sr_return_amt#208, sr_net_loss#209] -Input [11]: [ss_item_sk#199, ss_store_sk#200, ss_promo_sk#201, ss_ticket_number#202, ss_ext_sales_price#203, ss_net_profit#204, ss_sold_date_sk#205, sr_item_sk#206, sr_ticket_number#207, sr_return_amt#208, sr_net_loss#209] +Output [8]: [ss_item_sk#205, ss_store_sk#206, ss_promo_sk#207, ss_ext_sales_price#209, ss_net_profit#210, ss_sold_date_sk#211, sr_return_amt#214, sr_net_loss#215] +Input [11]: [ss_item_sk#205, ss_store_sk#206, ss_promo_sk#207, ss_ticket_number#208, ss_ext_sales_price#209, ss_net_profit#210, ss_sold_date_sk#211, sr_item_sk#212, sr_ticket_number#213, sr_return_amt#214, sr_net_loss#215] (150) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#210] +Output [1]: [i_item_sk#216] (151) BroadcastHashJoin [codegen id : 74] -Left keys [1]: [ss_item_sk#199] -Right keys [1]: [i_item_sk#210] +Left keys [1]: [ss_item_sk#205] +Right keys [1]: [i_item_sk#216] Join type: Inner Join condition: None (152) Project [codegen id : 74] -Output [7]: [ss_store_sk#200, ss_promo_sk#201, ss_ext_sales_price#203, ss_net_profit#204, ss_sold_date_sk#205, sr_return_amt#208, sr_net_loss#209] -Input [9]: [ss_item_sk#199, ss_store_sk#200, ss_promo_sk#201, ss_ext_sales_price#203, ss_net_profit#204, ss_sold_date_sk#205, sr_return_amt#208, sr_net_loss#209, i_item_sk#210] +Output [7]: [ss_store_sk#206, ss_promo_sk#207, ss_ext_sales_price#209, ss_net_profit#210, ss_sold_date_sk#211, sr_return_amt#214, sr_net_loss#215] +Input [9]: [ss_item_sk#205, ss_store_sk#206, ss_promo_sk#207, ss_ext_sales_price#209, ss_net_profit#210, ss_sold_date_sk#211, sr_return_amt#214, sr_net_loss#215, i_item_sk#216] (153) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#211] +Output [1]: [p_promo_sk#217] (154) BroadcastHashJoin [codegen id : 74] -Left keys [1]: [ss_promo_sk#201] -Right keys [1]: [p_promo_sk#211] +Left keys [1]: [ss_promo_sk#207] +Right keys [1]: [p_promo_sk#217] Join type: Inner Join condition: None (155) Project [codegen id : 74] -Output [6]: [ss_store_sk#200, ss_ext_sales_price#203, ss_net_profit#204, ss_sold_date_sk#205, sr_return_amt#208, sr_net_loss#209] -Input [8]: [ss_store_sk#200, ss_promo_sk#201, ss_ext_sales_price#203, ss_net_profit#204, ss_sold_date_sk#205, sr_return_amt#208, sr_net_loss#209, p_promo_sk#211] +Output [6]: [ss_store_sk#206, ss_ext_sales_price#209, ss_net_profit#210, ss_sold_date_sk#211, sr_return_amt#214, sr_net_loss#215] +Input [8]: [ss_store_sk#206, ss_promo_sk#207, ss_ext_sales_price#209, ss_net_profit#210, ss_sold_date_sk#211, sr_return_amt#214, sr_net_loss#215, p_promo_sk#217] (156) ReusedExchange [Reuses operator id: 221] -Output [1]: [d_date_sk#212] +Output [1]: [d_date_sk#218] (157) BroadcastHashJoin [codegen id : 74] -Left keys [1]: [ss_sold_date_sk#205] -Right keys [1]: [d_date_sk#212] +Left keys [1]: [ss_sold_date_sk#211] +Right keys [1]: [d_date_sk#218] Join type: Inner Join condition: None (158) Project [codegen id : 74] -Output [5]: [ss_store_sk#200, ss_ext_sales_price#203, ss_net_profit#204, sr_return_amt#208, sr_net_loss#209] -Input [7]: [ss_store_sk#200, ss_ext_sales_price#203, ss_net_profit#204, ss_sold_date_sk#205, sr_return_amt#208, sr_net_loss#209, d_date_sk#212] +Output [5]: [ss_store_sk#206, ss_ext_sales_price#209, ss_net_profit#210, sr_return_amt#214, sr_net_loss#215] +Input [7]: [ss_store_sk#206, ss_ext_sales_price#209, ss_net_profit#210, ss_sold_date_sk#211, sr_return_amt#214, sr_net_loss#215, d_date_sk#218] (159) ReusedExchange [Reuses operator id: 34] -Output [2]: [s_store_sk#213, s_store_id#214] +Output [2]: [s_store_sk#219, s_store_id#220] (160) BroadcastHashJoin [codegen id : 74] -Left keys [1]: [ss_store_sk#200] -Right keys [1]: [s_store_sk#213] +Left keys [1]: [ss_store_sk#206] +Right keys [1]: [s_store_sk#219] Join type: Inner Join condition: None (161) Project [codegen id : 74] -Output [5]: [ss_ext_sales_price#203, ss_net_profit#204, sr_return_amt#208, sr_net_loss#209, s_store_id#214] -Input [7]: [ss_store_sk#200, ss_ext_sales_price#203, ss_net_profit#204, sr_return_amt#208, sr_net_loss#209, s_store_sk#213, s_store_id#214] +Output [5]: [ss_ext_sales_price#209, ss_net_profit#210, sr_return_amt#214, sr_net_loss#215, s_store_id#220] +Input [7]: [ss_store_sk#206, ss_ext_sales_price#209, ss_net_profit#210, sr_return_amt#214, sr_net_loss#215, s_store_sk#219, s_store_id#220] (162) HashAggregate [codegen id : 74] -Input [5]: [ss_ext_sales_price#203, ss_net_profit#204, sr_return_amt#208, sr_net_loss#209, s_store_id#214] -Keys [1]: [s_store_id#214] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#203)), partial_sum(coalesce(cast(sr_return_amt#208 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#204 - coalesce(cast(sr_net_loss#209 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#215, sum#216, isEmpty#217, sum#218, isEmpty#219] -Results [6]: [s_store_id#214, sum#220, sum#221, isEmpty#222, sum#223, isEmpty#224] +Input [5]: [ss_ext_sales_price#209, ss_net_profit#210, sr_return_amt#214, sr_net_loss#215, s_store_id#220] +Keys [1]: [s_store_id#220] +Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#209)), partial_sum(coalesce(cast(sr_return_amt#214 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#210 - coalesce(cast(sr_net_loss#215 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#221, sum#222, isEmpty#223, sum#224, isEmpty#225] +Results [6]: [s_store_id#220, sum#226, sum#227, isEmpty#228, sum#229, isEmpty#230] (163) Exchange -Input [6]: [s_store_id#214, sum#220, sum#221, isEmpty#222, sum#223, isEmpty#224] -Arguments: hashpartitioning(s_store_id#214, 5), ENSURE_REQUIREMENTS, [plan_id=21] +Input [6]: [s_store_id#220, sum#226, sum#227, isEmpty#228, sum#229, isEmpty#230] +Arguments: hashpartitioning(s_store_id#220, 5), ENSURE_REQUIREMENTS, [plan_id=21] (164) HashAggregate [codegen id : 75] -Input [6]: [s_store_id#214, sum#220, sum#221, isEmpty#222, sum#223, isEmpty#224] -Keys [1]: [s_store_id#214] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#203)), sum(coalesce(cast(sr_return_amt#208 as decimal(12,2)), 0.00)), sum((ss_net_profit#204 - coalesce(cast(sr_net_loss#209 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#203))#35, sum(coalesce(cast(sr_return_amt#208 as decimal(12,2)), 0.00))#36, sum((ss_net_profit#204 - coalesce(cast(sr_net_loss#209 as decimal(12,2)), 0.00)))#37] -Results [5]: [store channel AS channel#38, concat(store, s_store_id#214) AS id#39, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#203))#35,17,2) AS sales#40, sum(coalesce(cast(sr_return_amt#208 as decimal(12,2)), 0.00))#36 AS returns#41, sum((ss_net_profit#204 - coalesce(cast(sr_net_loss#209 as decimal(12,2)), 0.00)))#37 AS profit#42] +Input [6]: [s_store_id#220, sum#226, sum#227, isEmpty#228, sum#229, isEmpty#230] +Keys [1]: [s_store_id#220] +Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#209)), sum(coalesce(cast(sr_return_amt#214 as decimal(12,2)), 0.00)), sum((ss_net_profit#210 - coalesce(cast(sr_net_loss#215 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#209))#35, sum(coalesce(cast(sr_return_amt#214 as decimal(12,2)), 0.00))#36, sum((ss_net_profit#210 - coalesce(cast(sr_net_loss#215 as decimal(12,2)), 0.00)))#37] +Results [5]: [store channel AS channel#231, concat(store, s_store_id#220) AS id#232, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#209))#35,17,2) AS sales#40, sum(coalesce(cast(sr_return_amt#214 as decimal(12,2)), 0.00))#36 AS returns#41, sum((ss_net_profit#210 - coalesce(cast(sr_net_loss#215 as decimal(12,2)), 0.00)))#37 AS profit#42] (165) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#225, cs_item_sk#226, cs_promo_sk#227, cs_order_number#228, cs_ext_sales_price#229, cs_net_profit#230, cs_sold_date_sk#231] +Output [7]: [cs_catalog_page_sk#233, cs_item_sk#234, cs_promo_sk#235, cs_order_number#236, cs_ext_sales_price#237, cs_net_profit#238, cs_sold_date_sk#239] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#231), dynamicpruningexpression(cs_sold_date_sk#231 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#239), dynamicpruningexpression(cs_sold_date_sk#239 IN dynamicpruning#8)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct (166) ColumnarToRow [codegen id : 76] -Input [7]: [cs_catalog_page_sk#225, cs_item_sk#226, cs_promo_sk#227, cs_order_number#228, cs_ext_sales_price#229, cs_net_profit#230, cs_sold_date_sk#231] +Input [7]: [cs_catalog_page_sk#233, cs_item_sk#234, cs_promo_sk#235, cs_order_number#236, cs_ext_sales_price#237, cs_net_profit#238, cs_sold_date_sk#239] (167) Filter [codegen id : 76] -Input [7]: [cs_catalog_page_sk#225, cs_item_sk#226, cs_promo_sk#227, cs_order_number#228, cs_ext_sales_price#229, cs_net_profit#230, cs_sold_date_sk#231] -Condition : ((isnotnull(cs_catalog_page_sk#225) AND isnotnull(cs_item_sk#226)) AND isnotnull(cs_promo_sk#227)) +Input [7]: [cs_catalog_page_sk#233, cs_item_sk#234, cs_promo_sk#235, cs_order_number#236, cs_ext_sales_price#237, cs_net_profit#238, cs_sold_date_sk#239] +Condition : ((isnotnull(cs_catalog_page_sk#233) AND isnotnull(cs_item_sk#234)) AND isnotnull(cs_promo_sk#235)) (168) Exchange -Input [7]: [cs_catalog_page_sk#225, cs_item_sk#226, cs_promo_sk#227, cs_order_number#228, cs_ext_sales_price#229, cs_net_profit#230, cs_sold_date_sk#231] -Arguments: hashpartitioning(cs_item_sk#226, cs_order_number#228, 5), ENSURE_REQUIREMENTS, [plan_id=22] +Input [7]: [cs_catalog_page_sk#233, cs_item_sk#234, cs_promo_sk#235, cs_order_number#236, cs_ext_sales_price#237, cs_net_profit#238, cs_sold_date_sk#239] +Arguments: hashpartitioning(cs_item_sk#234, cs_order_number#236, 5), ENSURE_REQUIREMENTS, [plan_id=22] (169) Sort [codegen id : 77] -Input [7]: [cs_catalog_page_sk#225, cs_item_sk#226, cs_promo_sk#227, cs_order_number#228, cs_ext_sales_price#229, cs_net_profit#230, cs_sold_date_sk#231] -Arguments: [cs_item_sk#226 ASC NULLS FIRST, cs_order_number#228 ASC NULLS FIRST], false, 0 +Input [7]: [cs_catalog_page_sk#233, cs_item_sk#234, cs_promo_sk#235, cs_order_number#236, cs_ext_sales_price#237, cs_net_profit#238, cs_sold_date_sk#239] +Arguments: [cs_item_sk#234 ASC NULLS FIRST, cs_order_number#236 ASC NULLS FIRST], false, 0 (170) ReusedExchange [Reuses operator id: 49] -Output [4]: [cr_item_sk#232, cr_order_number#233, cr_return_amount#234, cr_net_loss#235] +Output [4]: [cr_item_sk#240, cr_order_number#241, cr_return_amount#242, cr_net_loss#243] (171) Sort [codegen id : 79] -Input [4]: [cr_item_sk#232, cr_order_number#233, cr_return_amount#234, cr_net_loss#235] -Arguments: [cr_item_sk#232 ASC NULLS FIRST, cr_order_number#233 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#240, cr_order_number#241, cr_return_amount#242, cr_net_loss#243] +Arguments: [cr_item_sk#240 ASC NULLS FIRST, cr_order_number#241 ASC NULLS FIRST], false, 0 (172) SortMergeJoin [codegen id : 84] -Left keys [2]: [cs_item_sk#226, cs_order_number#228] -Right keys [2]: [cr_item_sk#232, cr_order_number#233] +Left keys [2]: [cs_item_sk#234, cs_order_number#236] +Right keys [2]: [cr_item_sk#240, cr_order_number#241] Join type: LeftOuter Join condition: None (173) Project [codegen id : 84] -Output [8]: [cs_catalog_page_sk#225, cs_item_sk#226, cs_promo_sk#227, cs_ext_sales_price#229, cs_net_profit#230, cs_sold_date_sk#231, cr_return_amount#234, cr_net_loss#235] -Input [11]: [cs_catalog_page_sk#225, cs_item_sk#226, cs_promo_sk#227, cs_order_number#228, cs_ext_sales_price#229, cs_net_profit#230, cs_sold_date_sk#231, cr_item_sk#232, cr_order_number#233, cr_return_amount#234, cr_net_loss#235] +Output [8]: [cs_catalog_page_sk#233, cs_item_sk#234, cs_promo_sk#235, cs_ext_sales_price#237, cs_net_profit#238, cs_sold_date_sk#239, cr_return_amount#242, cr_net_loss#243] +Input [11]: [cs_catalog_page_sk#233, cs_item_sk#234, cs_promo_sk#235, cs_order_number#236, cs_ext_sales_price#237, cs_net_profit#238, cs_sold_date_sk#239, cr_item_sk#240, cr_order_number#241, cr_return_amount#242, cr_net_loss#243] (174) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#236] +Output [1]: [i_item_sk#244] (175) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [cs_item_sk#226] -Right keys [1]: [i_item_sk#236] +Left keys [1]: [cs_item_sk#234] +Right keys [1]: [i_item_sk#244] Join type: Inner Join condition: None (176) Project [codegen id : 84] -Output [7]: [cs_catalog_page_sk#225, cs_promo_sk#227, cs_ext_sales_price#229, cs_net_profit#230, cs_sold_date_sk#231, cr_return_amount#234, cr_net_loss#235] -Input [9]: [cs_catalog_page_sk#225, cs_item_sk#226, cs_promo_sk#227, cs_ext_sales_price#229, cs_net_profit#230, cs_sold_date_sk#231, cr_return_amount#234, cr_net_loss#235, i_item_sk#236] +Output [7]: [cs_catalog_page_sk#233, cs_promo_sk#235, cs_ext_sales_price#237, cs_net_profit#238, cs_sold_date_sk#239, cr_return_amount#242, cr_net_loss#243] +Input [9]: [cs_catalog_page_sk#233, cs_item_sk#234, cs_promo_sk#235, cs_ext_sales_price#237, cs_net_profit#238, cs_sold_date_sk#239, cr_return_amount#242, cr_net_loss#243, i_item_sk#244] (177) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#237] +Output [1]: [p_promo_sk#245] (178) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [cs_promo_sk#227] -Right keys [1]: [p_promo_sk#237] +Left keys [1]: [cs_promo_sk#235] +Right keys [1]: [p_promo_sk#245] Join type: Inner Join condition: None (179) Project [codegen id : 84] -Output [6]: [cs_catalog_page_sk#225, cs_ext_sales_price#229, cs_net_profit#230, cs_sold_date_sk#231, cr_return_amount#234, cr_net_loss#235] -Input [8]: [cs_catalog_page_sk#225, cs_promo_sk#227, cs_ext_sales_price#229, cs_net_profit#230, cs_sold_date_sk#231, cr_return_amount#234, cr_net_loss#235, p_promo_sk#237] +Output [6]: [cs_catalog_page_sk#233, cs_ext_sales_price#237, cs_net_profit#238, cs_sold_date_sk#239, cr_return_amount#242, cr_net_loss#243] +Input [8]: [cs_catalog_page_sk#233, cs_promo_sk#235, cs_ext_sales_price#237, cs_net_profit#238, cs_sold_date_sk#239, cr_return_amount#242, cr_net_loss#243, p_promo_sk#245] (180) ReusedExchange [Reuses operator id: 221] -Output [1]: [d_date_sk#238] +Output [1]: [d_date_sk#246] (181) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [cs_sold_date_sk#231] -Right keys [1]: [d_date_sk#238] +Left keys [1]: [cs_sold_date_sk#239] +Right keys [1]: [d_date_sk#246] Join type: Inner Join condition: None (182) Project [codegen id : 84] -Output [5]: [cs_catalog_page_sk#225, cs_ext_sales_price#229, cs_net_profit#230, cr_return_amount#234, cr_net_loss#235] -Input [7]: [cs_catalog_page_sk#225, cs_ext_sales_price#229, cs_net_profit#230, cs_sold_date_sk#231, cr_return_amount#234, cr_net_loss#235, d_date_sk#238] +Output [5]: [cs_catalog_page_sk#233, cs_ext_sales_price#237, cs_net_profit#238, cr_return_amount#242, cr_net_loss#243] +Input [7]: [cs_catalog_page_sk#233, cs_ext_sales_price#237, cs_net_profit#238, cs_sold_date_sk#239, cr_return_amount#242, cr_net_loss#243, d_date_sk#246] (183) ReusedExchange [Reuses operator id: 65] -Output [2]: [cp_catalog_page_sk#239, cp_catalog_page_id#240] +Output [2]: [cp_catalog_page_sk#247, cp_catalog_page_id#248] (184) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [cs_catalog_page_sk#225] -Right keys [1]: [cp_catalog_page_sk#239] +Left keys [1]: [cs_catalog_page_sk#233] +Right keys [1]: [cp_catalog_page_sk#247] Join type: Inner Join condition: None (185) Project [codegen id : 84] -Output [5]: [cs_ext_sales_price#229, cs_net_profit#230, cr_return_amount#234, cr_net_loss#235, cp_catalog_page_id#240] -Input [7]: [cs_catalog_page_sk#225, cs_ext_sales_price#229, cs_net_profit#230, cr_return_amount#234, cr_net_loss#235, cp_catalog_page_sk#239, cp_catalog_page_id#240] +Output [5]: [cs_ext_sales_price#237, cs_net_profit#238, cr_return_amount#242, cr_net_loss#243, cp_catalog_page_id#248] +Input [7]: [cs_catalog_page_sk#233, cs_ext_sales_price#237, cs_net_profit#238, cr_return_amount#242, cr_net_loss#243, cp_catalog_page_sk#247, cp_catalog_page_id#248] (186) HashAggregate [codegen id : 84] -Input [5]: [cs_ext_sales_price#229, cs_net_profit#230, cr_return_amount#234, cr_net_loss#235, cp_catalog_page_id#240] -Keys [1]: [cp_catalog_page_id#240] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#229)), partial_sum(coalesce(cast(cr_return_amount#234 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#230 - coalesce(cast(cr_net_loss#235 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#241, sum#242, isEmpty#243, sum#244, isEmpty#245] -Results [6]: [cp_catalog_page_id#240, sum#246, sum#247, isEmpty#248, sum#249, isEmpty#250] +Input [5]: [cs_ext_sales_price#237, cs_net_profit#238, cr_return_amount#242, cr_net_loss#243, cp_catalog_page_id#248] +Keys [1]: [cp_catalog_page_id#248] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#237)), partial_sum(coalesce(cast(cr_return_amount#242 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#238 - coalesce(cast(cr_net_loss#243 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#249, sum#250, isEmpty#251, sum#252, isEmpty#253] +Results [6]: [cp_catalog_page_id#248, sum#254, sum#255, isEmpty#256, sum#257, isEmpty#258] (187) Exchange -Input [6]: [cp_catalog_page_id#240, sum#246, sum#247, isEmpty#248, sum#249, isEmpty#250] -Arguments: hashpartitioning(cp_catalog_page_id#240, 5), ENSURE_REQUIREMENTS, [plan_id=23] +Input [6]: [cp_catalog_page_id#248, sum#254, sum#255, isEmpty#256, sum#257, isEmpty#258] +Arguments: hashpartitioning(cp_catalog_page_id#248, 5), ENSURE_REQUIREMENTS, [plan_id=23] (188) HashAggregate [codegen id : 85] -Input [6]: [cp_catalog_page_id#240, sum#246, sum#247, isEmpty#248, sum#249, isEmpty#250] -Keys [1]: [cp_catalog_page_id#240] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#229)), sum(coalesce(cast(cr_return_amount#234 as decimal(12,2)), 0.00)), sum((cs_net_profit#230 - coalesce(cast(cr_net_loss#235 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#229))#70, sum(coalesce(cast(cr_return_amount#234 as decimal(12,2)), 0.00))#71, sum((cs_net_profit#230 - coalesce(cast(cr_net_loss#235 as decimal(12,2)), 0.00)))#72] -Results [5]: [catalog channel AS channel#73, concat(catalog_page, cp_catalog_page_id#240) AS id#74, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#229))#70,17,2) AS sales#75, sum(coalesce(cast(cr_return_amount#234 as decimal(12,2)), 0.00))#71 AS returns#76, sum((cs_net_profit#230 - coalesce(cast(cr_net_loss#235 as decimal(12,2)), 0.00)))#72 AS profit#77] +Input [6]: [cp_catalog_page_id#248, sum#254, sum#255, isEmpty#256, sum#257, isEmpty#258] +Keys [1]: [cp_catalog_page_id#248] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#237)), sum(coalesce(cast(cr_return_amount#242 as decimal(12,2)), 0.00)), sum((cs_net_profit#238 - coalesce(cast(cr_net_loss#243 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#237))#70, sum(coalesce(cast(cr_return_amount#242 as decimal(12,2)), 0.00))#71, sum((cs_net_profit#238 - coalesce(cast(cr_net_loss#243 as decimal(12,2)), 0.00)))#72] +Results [5]: [catalog channel AS channel#259, concat(catalog_page, cp_catalog_page_id#248) AS id#260, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#237))#70,17,2) AS sales#75, sum(coalesce(cast(cr_return_amount#242 as decimal(12,2)), 0.00))#71 AS returns#76, sum((cs_net_profit#238 - coalesce(cast(cr_net_loss#243 as decimal(12,2)), 0.00)))#72 AS profit#77] (189) ReusedExchange [Reuses operator id: 132] -Output [6]: [web_site_id#251, sum#252, sum#253, isEmpty#254, sum#255, isEmpty#256] +Output [6]: [web_site_id#261, sum#262, sum#263, isEmpty#264, sum#265, isEmpty#266] (190) HashAggregate [codegen id : 95] -Input [6]: [web_site_id#251, sum#252, sum#253, isEmpty#254, sum#255, isEmpty#256] -Keys [1]: [web_site_id#251] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#257)), sum(coalesce(cast(wr_return_amt#258 as decimal(12,2)), 0.00)), sum((ws_net_profit#259 - coalesce(cast(wr_net_loss#260 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#257))#105, sum(coalesce(cast(wr_return_amt#258 as decimal(12,2)), 0.00))#106, sum((ws_net_profit#259 - coalesce(cast(wr_net_loss#260 as decimal(12,2)), 0.00)))#107] -Results [5]: [web channel AS channel#108, concat(web_site, web_site_id#251) AS id#109, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#257))#105,17,2) AS sales#110, sum(coalesce(cast(wr_return_amt#258 as decimal(12,2)), 0.00))#106 AS returns#111, sum((ws_net_profit#259 - coalesce(cast(wr_net_loss#260 as decimal(12,2)), 0.00)))#107 AS profit#112] +Input [6]: [web_site_id#261, sum#262, sum#263, isEmpty#264, sum#265, isEmpty#266] +Keys [1]: [web_site_id#261] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#267)), sum(coalesce(cast(wr_return_amt#268 as decimal(12,2)), 0.00)), sum((ws_net_profit#269 - coalesce(cast(wr_net_loss#270 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#267))#105, sum(coalesce(cast(wr_return_amt#268 as decimal(12,2)), 0.00))#106, sum((ws_net_profit#269 - coalesce(cast(wr_net_loss#270 as decimal(12,2)), 0.00)))#107] +Results [5]: [web channel AS channel#271, concat(web_site, web_site_id#261) AS id#272, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#267))#105,17,2) AS sales#110, sum(coalesce(cast(wr_return_amt#268 as decimal(12,2)), 0.00))#106 AS returns#111, sum((ws_net_profit#269 - coalesce(cast(wr_net_loss#270 as decimal(12,2)), 0.00)))#107 AS profit#112] (191) Union (192) HashAggregate [codegen id : 96] -Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] -Keys [2]: [channel#38, id#39] +Input [5]: [channel#231, id#232, sales#40, returns#41, profit#42] +Keys [2]: [channel#231, id#232] Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] Aggregate Attributes [6]: [sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] -Results [8]: [channel#38, id#39, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Results [8]: [channel#231, id#232, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] (193) Exchange -Input [8]: [channel#38, id#39, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, [plan_id=24] +Input [8]: [channel#231, id#232, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Arguments: hashpartitioning(channel#231, id#232, 5), ENSURE_REQUIREMENTS, [plan_id=24] (194) HashAggregate [codegen id : 97] -Input [8]: [channel#38, id#39, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Keys [2]: [channel#38, id#39] +Input [8]: [channel#231, id#232, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Keys [2]: [channel#231, id#232] Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] Aggregate Attributes [3]: [sum(sales#40)#125, sum(returns#41)#126, sum(profit#42)#127] -Results [3]: [sum(sales#40)#125 AS sales#261, sum(returns#41)#126 AS returns#262, sum(profit#42)#127 AS profit#263] +Results [3]: [sum(sales#40)#125 AS sales#273, sum(returns#41)#126 AS returns#274, sum(profit#42)#127 AS profit#275] (195) HashAggregate [codegen id : 97] -Input [3]: [sales#261, returns#262, profit#263] +Input [3]: [sales#273, returns#274, profit#275] Keys: [] -Functions [3]: [partial_sum(sales#261), partial_sum(returns#262), partial_sum(profit#263)] -Aggregate Attributes [6]: [sum#264, isEmpty#265, sum#266, isEmpty#267, sum#268, isEmpty#269] -Results [6]: [sum#270, isEmpty#271, sum#272, isEmpty#273, sum#274, isEmpty#275] +Functions [3]: [partial_sum(sales#273), partial_sum(returns#274), partial_sum(profit#275)] +Aggregate Attributes [6]: [sum#276, isEmpty#277, sum#278, isEmpty#279, sum#280, isEmpty#281] +Results [6]: [sum#282, isEmpty#283, sum#284, isEmpty#285, sum#286, isEmpty#287] (196) Exchange -Input [6]: [sum#270, isEmpty#271, sum#272, isEmpty#273, sum#274, isEmpty#275] +Input [6]: [sum#282, isEmpty#283, sum#284, isEmpty#285, sum#286, isEmpty#287] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=25] (197) HashAggregate [codegen id : 98] -Input [6]: [sum#270, isEmpty#271, sum#272, isEmpty#273, sum#274, isEmpty#275] +Input [6]: [sum#282, isEmpty#283, sum#284, isEmpty#285, sum#286, isEmpty#287] Keys: [] -Functions [3]: [sum(sales#261), sum(returns#262), sum(profit#263)] -Aggregate Attributes [3]: [sum(sales#261)#276, sum(returns#262)#277, sum(profit#263)#278] -Results [5]: [null AS channel#279, null AS id#280, sum(sales#261)#276 AS sales#281, sum(returns#262)#277 AS returns#282, sum(profit#263)#278 AS profit#283] +Functions [3]: [sum(sales#273), sum(returns#274), sum(profit#275)] +Aggregate Attributes [3]: [sum(sales#273)#288, sum(returns#274)#289, sum(profit#275)#290] +Results [5]: [null AS channel#291, null AS id#292, sum(sales#273)#288 AS sales#293, sum(returns#274)#289 AS returns#294, sum(profit#275)#290 AS profit#295] (198) Union @@ -1187,19 +1187,19 @@ Input [2]: [i_item_sk#18, i_current_price#19] Input [1]: [i_item_sk#18] Keys: [] Functions [1]: [partial_bloom_filter_agg(xxhash64(i_item_sk#18, 42), 101823, 1521109, 0, 0)] -Aggregate Attributes [1]: [buf#284] -Results [1]: [buf#285] +Aggregate Attributes [1]: [buf#296] +Results [1]: [buf#297] (208) Exchange -Input [1]: [buf#285] +Input [1]: [buf#297] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=27] (209) ObjectHashAggregate -Input [1]: [buf#285] +Input [1]: [buf#297] Keys: [] Functions [1]: [bloom_filter_agg(xxhash64(i_item_sk#18, 42), 101823, 1521109, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(i_item_sk#18, 42), 101823, 1521109, 0, 0)#286] -Results [1]: [bloom_filter_agg(xxhash64(i_item_sk#18, 42), 101823, 1521109, 0, 0)#286 AS bloomFilter#287] +Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(i_item_sk#18, 42), 101823, 1521109, 0, 0)#298] +Results [1]: [bloom_filter_agg(xxhash64(i_item_sk#18, 42), 101823, 1521109, 0, 0)#298 AS bloomFilter#299] Subquery:2 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#11, [id=#12] ObjectHashAggregate (216) @@ -1233,19 +1233,19 @@ Input [2]: [p_promo_sk#20, p_channel_tv#21] Input [1]: [p_promo_sk#20] Keys: [] Functions [1]: [partial_bloom_filter_agg(xxhash64(p_promo_sk#20, 42), 986, 24246, 0, 0)] -Aggregate Attributes [1]: [buf#288] -Results [1]: [buf#289] +Aggregate Attributes [1]: [buf#300] +Results [1]: [buf#301] (215) Exchange -Input [1]: [buf#289] +Input [1]: [buf#301] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=28] (216) ObjectHashAggregate -Input [1]: [buf#289] +Input [1]: [buf#301] Keys: [] Functions [1]: [bloom_filter_agg(xxhash64(p_promo_sk#20, 42), 986, 24246, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(p_promo_sk#20, 42), 986, 24246, 0, 0)#290] -Results [1]: [bloom_filter_agg(xxhash64(p_promo_sk#20, 42), 986, 24246, 0, 0)#290 AS bloomFilter#291] +Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(p_promo_sk#20, 42), 986, 24246, 0, 0)#302] +Results [1]: [bloom_filter_agg(xxhash64(p_promo_sk#20, 42), 986, 24246, 0, 0)#302 AS bloomFilter#303] Subquery:3 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (221) @@ -1256,22 +1256,22 @@ BroadcastExchange (221) (217) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#292] +Output [2]: [d_date_sk#22, d_date#304] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct (218) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#292] +Input [2]: [d_date_sk#22, d_date#304] (219) Filter [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#292] -Condition : (((isnotnull(d_date#292) AND (d_date#292 >= 1998-08-04)) AND (d_date#292 <= 1998-09-03)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#22, d_date#304] +Condition : (((isnotnull(d_date#304) AND (d_date#304 >= 1998-08-04)) AND (d_date#304 <= 1998-09-03)) AND isnotnull(d_date_sk#22)) (220) Project [codegen id : 1] Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_date#292] +Input [2]: [d_date_sk#22, d_date#304] (221) BroadcastExchange Input [1]: [d_date_sk#22] @@ -1289,10 +1289,10 @@ Subquery:8 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery Subquery:9 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#8 -Subquery:10 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#157 IN dynamicpruning#8 +Subquery:10 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#161 IN dynamicpruning#8 -Subquery:11 Hosting operator id = 141 Hosting Expression = ss_sold_date_sk#205 IN dynamicpruning#8 +Subquery:11 Hosting operator id = 141 Hosting Expression = ss_sold_date_sk#211 IN dynamicpruning#8 -Subquery:12 Hosting operator id = 165 Hosting Expression = cs_sold_date_sk#231 IN dynamicpruning#8 +Subquery:12 Hosting operator id = 165 Hosting Expression = cs_sold_date_sk#239 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt index a0881e1685e4..6770bd4fd7f5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt @@ -612,60 +612,60 @@ Aggregate Attributes [3]: [sum(sales#36)#121, sum(returns#37)#122, sum(profit#38 Results [5]: [channel#34, id#35, cast(sum(sales#36)#121 as decimal(37,2)) AS sales#124, cast(sum(returns#37)#122 as decimal(38,2)) AS returns#125, cast(sum(profit#38)#123 as decimal(38,2)) AS profit#126] (106) ReusedExchange [Reuses operator id: 104] -Output [8]: [channel#34, id#35, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] +Output [8]: [channel#127, id#128, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] (107) HashAggregate [codegen id : 64] -Input [8]: [channel#34, id#35, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] -Keys [2]: [channel#34, id#35] +Input [8]: [channel#127, id#128, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] +Keys [2]: [channel#127, id#128] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] Aggregate Attributes [3]: [sum(sales#36)#121, sum(returns#37)#122, sum(profit#38)#123] -Results [4]: [channel#34, sum(sales#36)#121 AS sales#127, sum(returns#37)#122 AS returns#128, sum(profit#38)#123 AS profit#129] +Results [4]: [channel#127, sum(sales#36)#121 AS sales#129, sum(returns#37)#122 AS returns#130, sum(profit#38)#123 AS profit#131] (108) HashAggregate [codegen id : 64] -Input [4]: [channel#34, sales#127, returns#128, profit#129] -Keys [1]: [channel#34] -Functions [3]: [partial_sum(sales#127), partial_sum(returns#128), partial_sum(profit#129)] -Aggregate Attributes [6]: [sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Results [7]: [channel#34, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] +Input [4]: [channel#127, sales#129, returns#130, profit#131] +Keys [1]: [channel#127] +Functions [3]: [partial_sum(sales#129), partial_sum(returns#130), partial_sum(profit#131)] +Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] +Results [7]: [channel#127, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] (109) Exchange -Input [7]: [channel#34, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, [plan_id=16] +Input [7]: [channel#127, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Arguments: hashpartitioning(channel#127, 5), ENSURE_REQUIREMENTS, [plan_id=16] (110) HashAggregate [codegen id : 65] -Input [7]: [channel#34, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Keys [1]: [channel#34] -Functions [3]: [sum(sales#127), sum(returns#128), sum(profit#129)] -Aggregate Attributes [3]: [sum(sales#127)#142, sum(returns#128)#143, sum(profit#129)#144] -Results [5]: [channel#34, null AS id#145, sum(sales#127)#142 AS sales#146, sum(returns#128)#143 AS returns#147, sum(profit#129)#144 AS profit#148] +Input [7]: [channel#127, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Keys [1]: [channel#127] +Functions [3]: [sum(sales#129), sum(returns#130), sum(profit#131)] +Aggregate Attributes [3]: [sum(sales#129)#144, sum(returns#130)#145, sum(profit#131)#146] +Results [5]: [channel#127, null AS id#147, sum(sales#129)#144 AS sales#148, sum(returns#130)#145 AS returns#149, sum(profit#131)#146 AS profit#150] (111) ReusedExchange [Reuses operator id: 104] -Output [8]: [channel#34, id#35, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] +Output [8]: [channel#151, id#152, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] (112) HashAggregate [codegen id : 97] -Input [8]: [channel#34, id#35, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] -Keys [2]: [channel#34, id#35] +Input [8]: [channel#151, id#152, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] +Keys [2]: [channel#151, id#152] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] Aggregate Attributes [3]: [sum(sales#36)#121, sum(returns#37)#122, sum(profit#38)#123] -Results [3]: [sum(sales#36)#121 AS sales#149, sum(returns#37)#122 AS returns#150, sum(profit#38)#123 AS profit#151] +Results [3]: [sum(sales#36)#121 AS sales#153, sum(returns#37)#122 AS returns#154, sum(profit#38)#123 AS profit#155] (113) HashAggregate [codegen id : 97] -Input [3]: [sales#149, returns#150, profit#151] +Input [3]: [sales#153, returns#154, profit#155] Keys: [] -Functions [3]: [partial_sum(sales#149), partial_sum(returns#150), partial_sum(profit#151)] -Aggregate Attributes [6]: [sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] -Results [6]: [sum#158, isEmpty#159, sum#160, isEmpty#161, sum#162, isEmpty#163] +Functions [3]: [partial_sum(sales#153), partial_sum(returns#154), partial_sum(profit#155)] +Aggregate Attributes [6]: [sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Results [6]: [sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] (114) Exchange -Input [6]: [sum#158, isEmpty#159, sum#160, isEmpty#161, sum#162, isEmpty#163] +Input [6]: [sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17] (115) HashAggregate [codegen id : 98] -Input [6]: [sum#158, isEmpty#159, sum#160, isEmpty#161, sum#162, isEmpty#163] +Input [6]: [sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] Keys: [] -Functions [3]: [sum(sales#149), sum(returns#150), sum(profit#151)] -Aggregate Attributes [3]: [sum(sales#149)#164, sum(returns#150)#165, sum(profit#151)#166] -Results [5]: [null AS channel#167, null AS id#168, sum(sales#149)#164 AS sales#169, sum(returns#150)#165 AS returns#170, sum(profit#151)#166 AS profit#171] +Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] +Aggregate Attributes [3]: [sum(sales#153)#168, sum(returns#154)#169, sum(profit#155)#170] +Results [5]: [null AS channel#171, null AS id#172, sum(sales#153)#168 AS sales#173, sum(returns#154)#169 AS returns#174, sum(profit#155)#170 AS profit#175] (116) Union @@ -702,22 +702,22 @@ BroadcastExchange (125) (121) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#172] +Output [2]: [d_date_sk#14, d_date#176] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct (122) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#14, d_date#172] +Input [2]: [d_date_sk#14, d_date#176] (123) Filter [codegen id : 1] -Input [2]: [d_date_sk#14, d_date#172] -Condition : (((isnotnull(d_date#172) AND (d_date#172 >= 1998-08-04)) AND (d_date#172 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_date#176] +Condition : (((isnotnull(d_date#176) AND (d_date#176 >= 1998-08-04)) AND (d_date#176 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) (124) Project [codegen id : 1] Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_date#172] +Input [2]: [d_date_sk#14, d_date#176] (125) BroadcastExchange Input [1]: [d_date_sk#14]