diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index 965618ee827d1..99b1546f858c1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -47,23 +47,12 @@ class PartitionerAwareUnionRDDPartition( } } -/** - * Class representing an RDD that can take multiple RDDs partitioned by the same partitioner and - * unify them into a single RDD while preserving the partitioner. So m RDDs with p partitions each - * will be unified to a single RDD with p partitions and the same partitioner. The preferred - * location for each partition of the unified RDD will be the most common preferred location - * of the corresponding partitions of the parent RDDs. For example, location of partition 0 - * of the unified RDD will be where most of partition 0 of the parent RDDs are located. - */ private[spark] -class PartitionerAwareUnionRDD[T: ClassTag]( +abstract class PartitionerAwareUnionRDDBase[T: ClassTag]( sc: SparkContext, var rdds: Seq[RDD[T]] ) extends RDD[T](sc, rdds.map(x => new OneToOneDependency(x))) { - require(rdds.nonEmpty) - require(rdds.forall(_.partitioner.isDefined)) - require(rdds.flatMap(_.partitioner).toSet.size == 1, - "Parent RDDs have different partitioners: " + rdds.flatMap(_.partitioner)) + require(rdds.nonEmpty, "RDDs cannot be empty") override val partitioner = rdds.head.partitioner @@ -111,3 +100,49 @@ class PartitionerAwareUnionRDD[T: ClassTag]( rdd.context.getPreferredLocs(rdd, part.index).map(tl => tl.host) } } + +/** + * Class representing an RDD that can take multiple RDDs partitioned by the same partitioner and + * unify them into a single RDD while preserving the partitioner. So m RDDs with p partitions each + * will be unified to a single RDD with p partitions and the same partitioner. The preferred + * location for each partition of the unified RDD will be the most common preferred location + * of the corresponding partitions of the parent RDDs. For example, location of partition 0 + * of the unified RDD will be where most of partition 0 of the parent RDDs are located. + */ +private[spark] +class PartitionerAwareUnionRDD[T: ClassTag]( + sc: SparkContext, + var _rdds: Seq[RDD[T]] + ) extends PartitionerAwareUnionRDDBase(sc, _rdds) { + require(_rdds.forall(_.partitioner.isDefined)) + require(_rdds.flatMap(_.partitioner).toSet.size == 1, + "Parent RDDs have different partitioners: " + _rdds.flatMap(_.partitioner)) +} + +/** + * This is similar to [[PartitionerAwareUnionRDD]], but it doesn't require the parent RDDs + * to have defined partitioner and have the same partitioner if defined. + * It is because SQL's shuffle RDD's partitioner is not defined in `ShuffledRowRDD`. + * The actual partitioning is implemented in `ShuffleExchangeExec.prepareShuffleDependency`. + * + * Thus, this RDD doesn't check the partitioner of parent RDDs. Its correctness relies on the + * fact that the given RDDs are partitioned in the same way. So before using this RDD, you must + * ensure that all parent RDDs are partitioned correctly by checking their SQL output partitioning. + */ +private[spark] +class SQLPartitioningAwareUnionRDD[T: ClassTag]( + sc: SparkContext, + var _rdds: Seq[RDD[T]], + val numPartitions: Int + ) extends PartitionerAwareUnionRDDBase(sc, _rdds) { + require(partitioner.isEmpty || partitioner.get.numPartitions == numPartitions, + "Partitioner of parent RDDs does not match the number of partitions: " + + s"expected $numPartitions, but got ${partitioner.map(_.numPartitions).getOrElse("none")}") + + override def getPartitions: Array[Partition] = { + (0 until numPartitions).map { index => + new PartitionerAwareUnionRDDPartition(_rdds, index) + }.toArray + } +} + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f4aea61f585f7..b5975df168f5b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -6014,6 +6014,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val UNION_OUTPUT_PARTITIONING = + buildConf("spark.sql.unionOutputPartitioning") + .internal() + .doc("When set to true, the output partitioning of UnionExec will be the same as the " + + "input partitioning if its children have same partitioning. Otherwise, it will be a " + + "default partitioning.") + .version("4.1.0") + .booleanConf + .createWithDefault(true) + val LEGACY_PARSE_QUERY_WITHOUT_EOF = buildConf("spark.sql.legacy.parseQueryWithoutEof") .internal() .doc( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 70ade390c7336..78d7eb4ec1f51 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -25,7 +25,7 @@ import scala.concurrent.ExecutionContext import scala.concurrent.duration.Duration import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, SparkException, TaskContext} -import org.apache.spark.rdd.{EmptyRDD, PartitionwiseSampledRDD, RDD} +import org.apache.spark.rdd.{EmptyRDD, PartitionwiseSampledRDD, RDD, SQLPartitioningAwareUnionRDD} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences @@ -699,8 +699,80 @@ case class UnionExec(children: Seq[SparkPlan]) extends SparkPlan { } } - protected override def doExecute(): RDD[InternalRow] = - sparkContext.union(children.map(_.execute())) + /** + * Returns the output partitionings of the children, with the attributes converted to + * the first child's attributes at the same position. + */ + private def prepareOutputPartitioning(): Seq[Partitioning] = { + // Create a map of attributes from the other children to the first child. + val firstAttrs = children.head.output + val attributesMap = children.tail.map(_.output).map { otherAttrs => + otherAttrs.zip(firstAttrs).map { case (attr, firstAttr) => + attr -> firstAttr + }.toMap + } + + val partitionings = children.map(_.outputPartitioning) + val firstPartitioning = partitionings.head + val otherPartitionings = partitionings.tail + + val convertedOtherPartitionings = otherPartitionings.zipWithIndex.map { case (p, idx) => + val attributeMap = attributesMap(idx) + p match { + case e: Expression => + e.transform { + case a: Attribute if attributeMap.contains(a) => attributeMap(a) + }.asInstanceOf[Partitioning] + case _ => p + } + } + Seq(firstPartitioning) ++ convertedOtherPartitionings + } + + private def comparePartitioning(left: Partitioning, right: Partitioning): Boolean = { + (left, right) match { + case (SinglePartition, SinglePartition) => true + case (l: HashPartitioningLike, r: HashPartitioningLike) => l == r + // Note: two `RangePartitioning`s with even same ordering and number of partitions + // are not equal, because they might have different partition bounds. + case _ => false + } + } + + override def outputPartitioning: Partitioning = { + if (conf.getConf(SQLConf.UNION_OUTPUT_PARTITIONING)) { + val partitionings = prepareOutputPartitioning() + if (partitionings.forall(comparePartitioning(_, partitionings.head))) { + val partitioner = partitionings.head + + // Take the output attributes of this union and map the partitioner to them. + val attributeMap = children.head.output.zip(output).toMap + partitioner match { + case e: Expression => + e.transform { + case a: Attribute if attributeMap.contains(a) => attributeMap(a) + }.asInstanceOf[Partitioning] + case _ => partitioner + } + } else { + super.outputPartitioning + } + } else { + super.outputPartitioning + } + } + + protected override def doExecute(): RDD[InternalRow] = { + if (outputPartitioning.isInstanceOf[UnknownPartitioning]) { + sparkContext.union(children.map(_.execute())) + } else { + // This union has a known partitioning, i.e., its children have the same partitioning + // in semantics so this union can choose not to change the partitioning by using a + // custom partitioning aware union RDD. + val nonEmptyRdds = children.map(_.execute()).filter(!_.partitions.isEmpty) + new SQLPartitioningAwareUnionRDD(sparkContext, nonEmptyRdds, outputPartitioning.numPartitions) + } + } override def supportsColumnar: Boolean = children.forall(_.supportsColumnar) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index 075a67dbbaaf6..d2c4f70ff7777 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -1,67 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (16) - : : +- * BroadcastHashJoin LeftSemi BuildRight (15) - : : :- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet spark_catalog.default.item (7) - : : +- BroadcastExchange (14) - : : +- * Project (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet spark_catalog.default.item (10) - : +- BroadcastExchange (23) - : +- * Project (22) - : +- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet spark_catalog.default.customer_address (19) - :- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * Filter (31) - : : : : +- * ColumnarToRow (30) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Filter (46) - : : : +- * ColumnarToRow (45) - : : : +- Scan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) +TakeOrderedAndProject (62) ++- * HashAggregate (61) + +- * HashAggregate (60) + +- Union (59) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (16) + : : +- * BroadcastHashJoin LeftSemi BuildRight (15) + : : :- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet spark_catalog.default.item (7) + : : +- BroadcastExchange (14) + : : +- * Project (13) + : : +- * Filter (12) + : : +- * ColumnarToRow (11) + : : +- Scan parquet spark_catalog.default.item (10) + : +- BroadcastExchange (23) + : +- * Project (22) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet spark_catalog.default.customer_address (19) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * Filter (31) + : : : : +- * ColumnarToRow (30) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * Filter (46) + : : : +- * ColumnarToRow (45) + : : : +- Scan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) (1) Scan parquet spark_catalog.default.store_sales @@ -79,7 +78,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) -(4) ReusedExchange [Reuses operator id: 68] +(4) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -213,7 +212,7 @@ Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_da Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) -(32) ReusedExchange [Reuses operator id: 68] +(32) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#21] (33) BroadcastHashJoin [codegen id : 11] @@ -285,7 +284,7 @@ Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_da Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] Condition : (isnotnull(ws_bill_addr_sk#30) AND isnotnull(ws_item_sk#29)) -(47) ReusedExchange [Reuses operator id: 68] +(47) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#33] (48) BroadcastHashJoin [codegen id : 17] @@ -351,52 +350,48 @@ Functions [1]: [partial_sum(total_sales#16)] Aggregate Attributes [2]: [sum#41, isEmpty#42] Results [3]: [i_manufact_id#8, sum#43, isEmpty#44] -(61) Exchange -Input [3]: [i_manufact_id#8, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_manufact_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(62) HashAggregate [codegen id : 20] +(61) HashAggregate [codegen id : 19] Input [3]: [i_manufact_id#8, sum#43, isEmpty#44] Keys [1]: [i_manufact_id#8] Functions [1]: [sum(total_sales#16)] Aggregate Attributes [1]: [sum(total_sales#16)#45] Results [2]: [i_manufact_id#8, sum(total_sales#16)#45 AS total_sales#46] -(63) TakeOrderedAndProject +(62) TakeOrderedAndProject Input [2]: [i_manufact_id#8, total_sales#46] Arguments: 100, [total_sales#46 ASC NULLS FIRST], [i_manufact_id#8, total_sales#46] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * Project (67) - +- * Filter (66) - +- * ColumnarToRow (65) - +- Scan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (67) ++- * Project (66) + +- * Filter (65) + +- * ColumnarToRow (64) + +- Scan parquet spark_catalog.default.date_dim (63) -(64) Scan parquet spark_catalog.default.date_dim +(63) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#47, d_moy#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 1] +(64) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -(66) Filter [codegen id : 1] +(65) Filter [codegen id : 1] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1998)) AND (d_moy#48 = 5)) AND isnotnull(d_date_sk#6)) -(67) Project [codegen id : 1] +(66) Project [codegen id : 1] Output [1]: [d_date_sk#6] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -(68) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index e47174615570c..70139c5caa7d6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -1,105 +1,102 @@ TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (20) + WholeStageCodegen (19) HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_manufact_id] #1 - WholeStageCodegen (19) - HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #2 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Project [i_manufact_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_category,i_manufact_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - WholeStageCodegen (12) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #7 - WholeStageCodegen (11) - HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #4 - InputAdapter - ReusedExchange [ca_address_sk] #6 - WholeStageCodegen (18) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #8 - WholeStageCodegen (17) - HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #4 - InputAdapter - ReusedExchange [ca_address_sk] #6 + HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_manufact_id] #1 + WholeStageCodegen (5) + HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (3) + BroadcastHashJoin [i_manufact_id,i_manufact_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Project [i_manufact_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_category,i_manufact_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + Project [ca_address_sk] + Filter [ca_gmt_offset,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + WholeStageCodegen (12) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_manufact_id] #6 + WholeStageCodegen (11) + HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_manufact_id] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_ext_sales_price,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_addr_sk,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_manufact_id] #3 + InputAdapter + ReusedExchange [ca_address_sk] #5 + WholeStageCodegen (18) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_manufact_id] #7 + WholeStageCodegen (17) + HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk,ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_manufact_id] #3 + InputAdapter + ReusedExchange [ca_address_sk] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index eae0ce1329922..9b64c961a5e96 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -1,67 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * Project (10) - : : +- * Filter (9) - : : +- * ColumnarToRow (8) - : : +- Scan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * Project (20) - : +- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet spark_catalog.default.item (17) - :- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * Filter (31) - : : : : +- * ColumnarToRow (30) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Filter (46) - : : : +- * ColumnarToRow (45) - : : : +- Scan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) +TakeOrderedAndProject (62) ++- * HashAggregate (61) + +- * HashAggregate (60) + +- Union (59) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * Project (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * Filter (16) + : : +- * ColumnarToRow (15) + : : +- Scan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * Project (20) + : +- * Filter (19) + : +- * ColumnarToRow (18) + : +- Scan parquet spark_catalog.default.item (17) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * Filter (31) + : : : : +- * ColumnarToRow (30) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * Filter (46) + : : : +- * ColumnarToRow (45) + : : : +- Scan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) (1) Scan parquet spark_catalog.default.store_sales @@ -79,7 +78,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) -(4) ReusedExchange [Reuses operator id: 68] +(4) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -213,7 +212,7 @@ Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_da Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) -(32) ReusedExchange [Reuses operator id: 68] +(32) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#21] (33) BroadcastHashJoin [codegen id : 11] @@ -285,7 +284,7 @@ Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_da Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] Condition : (isnotnull(ws_bill_addr_sk#30) AND isnotnull(ws_item_sk#29)) -(47) ReusedExchange [Reuses operator id: 68] +(47) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#33] (48) BroadcastHashJoin [codegen id : 17] @@ -351,52 +350,48 @@ Functions [1]: [partial_sum(total_sales#16)] Aggregate Attributes [2]: [sum#41, isEmpty#42] Results [3]: [i_manufact_id#10, sum#43, isEmpty#44] -(61) Exchange -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(62) HashAggregate [codegen id : 20] +(61) HashAggregate [codegen id : 19] Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] Keys [1]: [i_manufact_id#10] Functions [1]: [sum(total_sales#16)] Aggregate Attributes [1]: [sum(total_sales#16)#45] Results [2]: [i_manufact_id#10, sum(total_sales#16)#45 AS total_sales#46] -(63) TakeOrderedAndProject +(62) TakeOrderedAndProject Input [2]: [i_manufact_id#10, total_sales#46] Arguments: 100, [total_sales#46 ASC NULLS FIRST], [i_manufact_id#10, total_sales#46] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * Project (67) - +- * Filter (66) - +- * ColumnarToRow (65) - +- Scan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (67) ++- * Project (66) + +- * Filter (65) + +- * ColumnarToRow (64) + +- Scan parquet spark_catalog.default.date_dim (63) -(64) Scan parquet spark_catalog.default.date_dim +(63) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#47, d_moy#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 1] +(64) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -(66) Filter [codegen id : 1] +(65) Filter [codegen id : 1] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1998)) AND (d_moy#48 = 5)) AND isnotnull(d_date_sk#6)) -(67) Project [codegen id : 1] +(66) Project [codegen id : 1] Output [1]: [d_date_sk#6] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -(68) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index 959f7932eb5ae..b423d06e2cc4b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -1,105 +1,102 @@ TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (20) + WholeStageCodegen (19) HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_manufact_id] #1 - WholeStageCodegen (19) - HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #2 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [i_manufact_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (12) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #7 - WholeStageCodegen (11) - HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 - WholeStageCodegen (18) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #8 - WholeStageCodegen (17) - HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_manufact_id] #1 + WholeStageCodegen (5) + HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [ca_address_sk] + Filter [ca_gmt_offset,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + BroadcastHashJoin [i_manufact_id,i_manufact_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [i_manufact_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_category,i_manufact_id] + WholeStageCodegen (12) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_manufact_id] #6 + WholeStageCodegen (11) + HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_addr_sk,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_manufact_id] #4 + WholeStageCodegen (18) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_manufact_id] #7 + WholeStageCodegen (17) + HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk,ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_manufact_id] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt index 4ffd467c2dbbe..02d0c1eebba83 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt @@ -1,90 +1,89 @@ == Physical Plan == -TakeOrderedAndProject (86) -+- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- Union (82) - :- * Project (27) - : +- * Filter (26) - : +- Window (25) - : +- * Sort (24) - : +- Window (23) - : +- * Sort (22) - : +- Exchange (21) - : +- * HashAggregate (20) - : +- Exchange (19) - : +- * HashAggregate (18) - : +- * Project (17) - : +- * SortMergeJoin Inner (16) - : :- * Sort (9) - : : +- Exchange (8) - : : +- * Project (7) - : : +- * BroadcastHashJoin Inner BuildRight (6) - : : :- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- ReusedExchange (5) - : +- * Sort (15) - : +- Exchange (14) - : +- * Project (13) - : +- * Filter (12) - : +- * ColumnarToRow (11) - : +- Scan parquet spark_catalog.default.web_returns (10) - :- * Project (54) - : +- * Filter (53) - : +- Window (52) - : +- * Sort (51) - : +- Window (50) - : +- * Sort (49) - : +- Exchange (48) - : +- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * SortMergeJoin Inner (43) - : :- * Sort (36) - : : +- Exchange (35) - : : +- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Project (31) - : : : +- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet spark_catalog.default.catalog_sales (28) - : : +- ReusedExchange (32) - : +- * Sort (42) - : +- Exchange (41) - : +- * Project (40) - : +- * Filter (39) - : +- * ColumnarToRow (38) - : +- Scan parquet spark_catalog.default.catalog_returns (37) - +- * Project (81) - +- * Filter (80) - +- Window (79) - +- * Sort (78) - +- Window (77) - +- * Sort (76) - +- Exchange (75) - +- * HashAggregate (74) - +- Exchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * SortMergeJoin Inner (70) - :- * Sort (63) - : +- Exchange (62) - : +- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * Project (58) - : : +- * Filter (57) - : : +- * ColumnarToRow (56) - : : +- Scan parquet spark_catalog.default.store_sales (55) - : +- ReusedExchange (59) - +- * Sort (69) - +- Exchange (68) - +- * Project (67) - +- * Filter (66) - +- * ColumnarToRow (65) - +- Scan parquet spark_catalog.default.store_returns (64) +TakeOrderedAndProject (85) ++- * HashAggregate (84) + +- * HashAggregate (83) + +- Union (82) + :- * Project (27) + : +- * Filter (26) + : +- Window (25) + : +- * Sort (24) + : +- Window (23) + : +- * Sort (22) + : +- Exchange (21) + : +- * HashAggregate (20) + : +- Exchange (19) + : +- * HashAggregate (18) + : +- * Project (17) + : +- * SortMergeJoin Inner (16) + : :- * Sort (9) + : : +- Exchange (8) + : : +- * Project (7) + : : +- * BroadcastHashJoin Inner BuildRight (6) + : : :- * Project (4) + : : : +- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet spark_catalog.default.web_sales (1) + : : +- ReusedExchange (5) + : +- * Sort (15) + : +- Exchange (14) + : +- * Project (13) + : +- * Filter (12) + : +- * ColumnarToRow (11) + : +- Scan parquet spark_catalog.default.web_returns (10) + :- * Project (54) + : +- * Filter (53) + : +- Window (52) + : +- * Sort (51) + : +- Window (50) + : +- * Sort (49) + : +- Exchange (48) + : +- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * SortMergeJoin Inner (43) + : :- * Sort (36) + : : +- Exchange (35) + : : +- * Project (34) + : : +- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Project (31) + : : : +- * Filter (30) + : : : +- * ColumnarToRow (29) + : : : +- Scan parquet spark_catalog.default.catalog_sales (28) + : : +- ReusedExchange (32) + : +- * Sort (42) + : +- Exchange (41) + : +- * Project (40) + : +- * Filter (39) + : +- * ColumnarToRow (38) + : +- Scan parquet spark_catalog.default.catalog_returns (37) + +- * Project (81) + +- * Filter (80) + +- Window (79) + +- * Sort (78) + +- Window (77) + +- * Sort (76) + +- Exchange (75) + +- * HashAggregate (74) + +- Exchange (73) + +- * HashAggregate (72) + +- * Project (71) + +- * SortMergeJoin Inner (70) + :- * Sort (63) + : +- Exchange (62) + : +- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * Project (58) + : : +- * Filter (57) + : : +- * ColumnarToRow (56) + : : +- Scan parquet spark_catalog.default.store_sales (55) + : +- ReusedExchange (59) + +- * Sort (69) + +- Exchange (68) + +- * Project (67) + +- * Filter (66) + +- * ColumnarToRow (65) + +- Scan parquet spark_catalog.default.store_returns (64) (1) Scan parquet spark_catalog.default.web_sales @@ -106,7 +105,7 @@ Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND Output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -(5) ReusedExchange [Reuses operator id: 91] +(5) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#8] (6) BroadcastHashJoin [codegen id : 2] @@ -228,7 +227,7 @@ Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AN Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -(32) ReusedExchange [Reuses operator id: 91] +(32) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#42] (33) BroadcastHashJoin [codegen id : 12] @@ -350,7 +349,7 @@ Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AN Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -(59) ReusedExchange [Reuses operator id: 91] +(59) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#76] (60) BroadcastHashJoin [codegen id : 22] @@ -462,52 +461,48 @@ Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(84) Exchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(85) HashAggregate [codegen id : 32] +(84) HashAggregate [codegen id : 31] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(86) TakeOrderedAndProject +(85) TakeOrderedAndProject Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (91) -+- * Project (90) - +- * Filter (89) - +- * ColumnarToRow (88) - +- Scan parquet spark_catalog.default.date_dim (87) +BroadcastExchange (90) ++- * Project (89) + +- * Filter (88) + +- * ColumnarToRow (87) + +- Scan parquet spark_catalog.default.date_dim (86) -(87) Scan parquet spark_catalog.default.date_dim +(86) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#8, d_year#104, d_moy#105] 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,12), IsNotNull(d_date_sk)] ReadSchema: struct -(88) ColumnarToRow [codegen id : 1] +(87) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#8, d_year#104, d_moy#105] -(89) Filter [codegen id : 1] +(88) Filter [codegen id : 1] Input [3]: [d_date_sk#8, d_year#104, d_moy#105] Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 = 2001)) AND (d_moy#105 = 12)) AND isnotnull(d_date_sk#8)) -(90) Project [codegen id : 1] +(89) Project [codegen id : 1] Output [1]: [d_date_sk#8] Input [3]: [d_date_sk#8, d_year#104, d_moy#105] -(91) BroadcastExchange +(90) BroadcastExchange Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/simplified.txt index 80d1661b033de..cc2ebb0c36ea8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/simplified.txt @@ -1,160 +1,157 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (32) + WholeStageCodegen (31) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (31) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (10) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (9) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (8) - Sort [return_ratio] - InputAdapter - Exchange #2 - WholeStageCodegen (7) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (6) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (10) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (9) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (8) + Sort [return_ratio] + InputAdapter + Exchange #1 + WholeStageCodegen (7) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ws_item_sk] #2 + WholeStageCodegen (6) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ws_order_number,ws_item_sk] InputAdapter - WholeStageCodegen (3) - Sort [ws_order_number,ws_item_sk] - InputAdapter - Exchange [ws_order_number,ws_item_sk] #4 - WholeStageCodegen (2) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + Exchange [ws_order_number,ws_item_sk] #3 + WholeStageCodegen (2) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #5 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + WholeStageCodegen (5) + Sort [wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (5) - Sort [wr_order_number,wr_item_sk] - InputAdapter - Exchange [wr_order_number,wr_item_sk] #6 - WholeStageCodegen (4) - Project [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - Filter [wr_return_amt,wr_order_number,wr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - WholeStageCodegen (20) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (19) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (18) - Sort [return_ratio] - InputAdapter - Exchange #7 - WholeStageCodegen (17) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #8 - WholeStageCodegen (16) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Exchange [wr_order_number,wr_item_sk] #5 + WholeStageCodegen (4) + Project [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + Filter [wr_return_amt,wr_order_number,wr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + WholeStageCodegen (20) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (19) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (18) + Sort [return_ratio] + InputAdapter + Exchange #6 + WholeStageCodegen (17) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #7 + WholeStageCodegen (16) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (13) + Sort [cs_order_number,cs_item_sk] InputAdapter - WholeStageCodegen (13) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #9 - WholeStageCodegen (12) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Exchange [cs_order_number,cs_item_sk] #8 + WholeStageCodegen (12) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #5 + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + WholeStageCodegen (15) + Sort [cr_order_number,cr_item_sk] InputAdapter - WholeStageCodegen (15) - Sort [cr_order_number,cr_item_sk] - InputAdapter - Exchange [cr_order_number,cr_item_sk] #10 - WholeStageCodegen (14) - Project [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - Filter [cr_return_amount,cr_order_number,cr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - WholeStageCodegen (30) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (29) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (28) - Sort [return_ratio] - InputAdapter - Exchange #11 - WholeStageCodegen (27) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ss_item_sk] #12 - WholeStageCodegen (26) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Exchange [cr_order_number,cr_item_sk] #9 + WholeStageCodegen (14) + Project [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + Filter [cr_return_amount,cr_order_number,cr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + WholeStageCodegen (30) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (29) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (28) + Sort [return_ratio] + InputAdapter + Exchange #10 + WholeStageCodegen (27) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ss_item_sk] #11 + WholeStageCodegen (26) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (23) + Sort [ss_ticket_number,ss_item_sk] InputAdapter - WholeStageCodegen (23) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #13 - WholeStageCodegen (22) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Exchange [ss_ticket_number,ss_item_sk] #12 + WholeStageCodegen (22) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #5 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + WholeStageCodegen (25) + Sort [sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (25) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #14 - WholeStageCodegen (24) - Project [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - Filter [sr_return_amt,sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + Exchange [sr_ticket_number,sr_item_sk] #13 + WholeStageCodegen (24) + Project [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + Filter [sr_return_amt,sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index 9eea658d789e4..a796c272c9acf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -1,81 +1,80 @@ == Physical Plan == -TakeOrderedAndProject (77) -+- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (24) - : +- * Filter (23) - : +- Window (22) - : +- * Sort (21) - : +- Window (20) - : +- * Sort (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- Exchange (16) - : +- * HashAggregate (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildLeft (10) - : : :- BroadcastExchange (5) - : : : +- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * Project (9) - : : +- * Filter (8) - : : +- * ColumnarToRow (7) - : : +- Scan parquet spark_catalog.default.web_returns (6) - : +- ReusedExchange (12) - :- * Project (48) - : +- * Filter (47) - : +- Window (46) - : +- * Sort (45) - : +- Window (44) - : +- * Sort (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildLeft (34) - : : :- BroadcastExchange (29) - : : : +- * Project (28) - : : : +- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet spark_catalog.default.catalog_sales (25) - : : +- * Project (33) - : : +- * Filter (32) - : : +- * ColumnarToRow (31) - : : +- Scan parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * Sort (67) - +- Exchange (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildLeft (58) - : :- BroadcastExchange (53) - : : +- * Project (52) - : : +- * Filter (51) - : : +- * ColumnarToRow (50) - : : +- Scan parquet spark_catalog.default.store_sales (49) - : +- * Project (57) - : +- * Filter (56) - : +- * ColumnarToRow (55) - : +- Scan parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (60) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (24) + : +- * Filter (23) + : +- Window (22) + : +- * Sort (21) + : +- Window (20) + : +- * Sort (19) + : +- Exchange (18) + : +- * HashAggregate (17) + : +- Exchange (16) + : +- * HashAggregate (15) + : +- * Project (14) + : +- * BroadcastHashJoin Inner BuildRight (13) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildLeft (10) + : : :- BroadcastExchange (5) + : : : +- * Project (4) + : : : +- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet spark_catalog.default.web_sales (1) + : : +- * Project (9) + : : +- * Filter (8) + : : +- * ColumnarToRow (7) + : : +- Scan parquet spark_catalog.default.web_returns (6) + : +- ReusedExchange (12) + :- * Project (48) + : +- * Filter (47) + : +- Window (46) + : +- * Sort (45) + : +- Window (44) + : +- * Sort (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (35) + : : +- * BroadcastHashJoin Inner BuildLeft (34) + : : :- BroadcastExchange (29) + : : : +- * Project (28) + : : : +- * Filter (27) + : : : +- * ColumnarToRow (26) + : : : +- Scan parquet spark_catalog.default.catalog_sales (25) + : : +- * Project (33) + : : +- * Filter (32) + : : +- * ColumnarToRow (31) + : : +- Scan parquet spark_catalog.default.catalog_returns (30) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * Sort (67) + +- Exchange (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * HashAggregate (63) + +- * Project (62) + +- * BroadcastHashJoin Inner BuildRight (61) + :- * Project (59) + : +- * BroadcastHashJoin Inner BuildLeft (58) + : :- BroadcastExchange (53) + : : +- * Project (52) + : : +- * Filter (51) + : : +- * ColumnarToRow (50) + : : +- Scan parquet spark_catalog.default.store_sales (49) + : +- * Project (57) + : +- * Filter (56) + : +- * ColumnarToRow (55) + : +- Scan parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (60) (1) Scan parquet spark_catalog.default.web_sales @@ -129,7 +128,7 @@ Join condition: None Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(12) ReusedExchange [Reuses operator id: 82] +(12) ReusedExchange [Reuses operator id: 81] Output [1]: [d_date_sk#13] (13) BroadcastHashJoin [codegen id : 3] @@ -239,7 +238,7 @@ Join condition: None Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(36) ReusedExchange [Reuses operator id: 82] +(36) ReusedExchange [Reuses operator id: 81] Output [1]: [d_date_sk#47] (37) BroadcastHashJoin [codegen id : 10] @@ -349,7 +348,7 @@ Join condition: None Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(60) ReusedExchange [Reuses operator id: 82] +(60) ReusedExchange [Reuses operator id: 81] Output [1]: [d_date_sk#81] (61) BroadcastHashJoin [codegen id : 17] @@ -417,52 +416,48 @@ Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(75) Exchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(76) HashAggregate [codegen id : 23] +(75) HashAggregate [codegen id : 22] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(77) TakeOrderedAndProject +(76) TakeOrderedAndProject Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (82) -+- * Project (81) - +- * Filter (80) - +- * ColumnarToRow (79) - +- Scan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (81) ++- * Project (80) + +- * Filter (79) + +- * ColumnarToRow (78) + +- Scan parquet spark_catalog.default.date_dim (77) -(78) Scan parquet spark_catalog.default.date_dim +(77) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#104, d_moy#105] 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,12), IsNotNull(d_date_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -(80) Filter [codegen id : 1] +(79) Filter [codegen id : 1] Input [3]: [d_date_sk#13, d_year#104, d_moy#105] Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 = 2001)) AND (d_moy#105 = 12)) AND isnotnull(d_date_sk#13)) -(81) Project [codegen id : 1] +(80) Project [codegen id : 1] Output [1]: [d_date_sk#13] Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -(82) BroadcastExchange +(81) BroadcastExchange Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index cfb4b948b4e8b..2ea310d01171d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -1,133 +1,130 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (23) + WholeStageCodegen (22) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - Sort [return_ratio] - InputAdapter - Exchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - Project [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - Filter [wr_return_amt,wr_order_number,wr_item_sk] + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (7) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (6) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (5) + Sort [return_ratio] + InputAdapter + Exchange #1 + WholeStageCodegen (4) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ws_item_sk] #2 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - Sort [return_ratio] - InputAdapter - Exchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + Project [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + Filter [wr_return_amt,wr_order_number,wr_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - Project [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - Filter [cr_return_amount,cr_order_number,cr_item_sk] + Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (14) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (13) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (12) + Sort [return_ratio] + InputAdapter + Exchange #5 + WholeStageCodegen (11) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #6 + WholeStageCodegen (10) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - Sort [return_ratio] - InputAdapter - Exchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + Project [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + Filter [cr_return_amount,cr_order_number,cr_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - Project [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - Filter [sr_return_amt,sr_ticket_number,sr_item_sk] + Scan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (21) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (20) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (19) + Sort [return_ratio] + InputAdapter + Exchange #8 + WholeStageCodegen (18) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ss_item_sk] #9 + WholeStageCodegen (17) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (15) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + Project [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + Filter [sr_return_amt,sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 66d33e5bd5464..daf0cb82cb736 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -1,67 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * Project (10) - : : +- * Filter (9) - : : +- * ColumnarToRow (8) - : : +- Scan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * Project (20) - : +- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet spark_catalog.default.item (17) - :- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * Filter (31) - : : : : +- * ColumnarToRow (30) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Filter (46) - : : : +- * ColumnarToRow (45) - : : : +- Scan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) +TakeOrderedAndProject (62) ++- * HashAggregate (61) + +- * HashAggregate (60) + +- Union (59) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * Project (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * Filter (16) + : : +- * ColumnarToRow (15) + : : +- Scan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * Project (20) + : +- * Filter (19) + : +- * ColumnarToRow (18) + : +- Scan parquet spark_catalog.default.item (17) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * Filter (31) + : : : : +- * ColumnarToRow (30) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * Filter (46) + : : : +- * ColumnarToRow (45) + : : : +- Scan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) (1) Scan parquet spark_catalog.default.store_sales @@ -79,7 +78,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) -(4) ReusedExchange [Reuses operator id: 68] +(4) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -213,7 +212,7 @@ Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_da Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) -(32) ReusedExchange [Reuses operator id: 68] +(32) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#21] (33) BroadcastHashJoin [codegen id : 11] @@ -285,7 +284,7 @@ Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_da Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] Condition : (isnotnull(ws_bill_addr_sk#30) AND isnotnull(ws_item_sk#29)) -(47) ReusedExchange [Reuses operator id: 68] +(47) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#33] (48) BroadcastHashJoin [codegen id : 17] @@ -351,52 +350,48 @@ Functions [1]: [partial_sum(total_sales#16)] Aggregate Attributes [2]: [sum#41, isEmpty#42] Results [3]: [i_item_id#10, sum#43, isEmpty#44] -(61) Exchange -Input [3]: [i_item_id#10, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(62) HashAggregate [codegen id : 20] +(61) HashAggregate [codegen id : 19] Input [3]: [i_item_id#10, sum#43, isEmpty#44] Keys [1]: [i_item_id#10] Functions [1]: [sum(total_sales#16)] Aggregate Attributes [1]: [sum(total_sales#16)#45] Results [2]: [i_item_id#10, sum(total_sales#16)#45 AS total_sales#46] -(63) TakeOrderedAndProject +(62) TakeOrderedAndProject Input [2]: [i_item_id#10, total_sales#46] Arguments: 100, [total_sales#46 ASC NULLS FIRST], [i_item_id#10, total_sales#46] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * Project (67) - +- * Filter (66) - +- * ColumnarToRow (65) - +- Scan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (67) ++- * Project (66) + +- * Filter (65) + +- * ColumnarToRow (64) + +- Scan parquet spark_catalog.default.date_dim (63) -(64) Scan parquet spark_catalog.default.date_dim +(63) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#47, d_moy#48] 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 -(65) ColumnarToRow [codegen id : 1] +(64) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -(66) Filter [codegen id : 1] +(65) Filter [codegen id : 1] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 2001)) AND (d_moy#48 = 2)) AND isnotnull(d_date_sk#6)) -(67) Project [codegen id : 1] +(66) Project [codegen id : 1] Output [1]: [d_date_sk#6] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -(68) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt index 4177a855c93a9..e15e0921767f5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt @@ -1,105 +1,102 @@ TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (20) + WholeStageCodegen (19) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [i_item_id] - Filter [i_color] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (11) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (17) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [ca_address_sk] + Filter [ca_gmt_offset,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_id,i_item_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [i_item_id] + Filter [i_color] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_id,i_color] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #6 + WholeStageCodegen (11) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_addr_sk,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 + WholeStageCodegen (18) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #7 + WholeStageCodegen (17) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk,ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 66d33e5bd5464..daf0cb82cb736 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -1,67 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * Project (10) - : : +- * Filter (9) - : : +- * ColumnarToRow (8) - : : +- Scan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * Project (20) - : +- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet spark_catalog.default.item (17) - :- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * Filter (31) - : : : : +- * ColumnarToRow (30) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Filter (46) - : : : +- * ColumnarToRow (45) - : : : +- Scan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) +TakeOrderedAndProject (62) ++- * HashAggregate (61) + +- * HashAggregate (60) + +- Union (59) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * Project (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * Filter (16) + : : +- * ColumnarToRow (15) + : : +- Scan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * Project (20) + : +- * Filter (19) + : +- * ColumnarToRow (18) + : +- Scan parquet spark_catalog.default.item (17) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * Filter (31) + : : : : +- * ColumnarToRow (30) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * Filter (46) + : : : +- * ColumnarToRow (45) + : : : +- Scan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) (1) Scan parquet spark_catalog.default.store_sales @@ -79,7 +78,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) -(4) ReusedExchange [Reuses operator id: 68] +(4) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -213,7 +212,7 @@ Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_da Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) -(32) ReusedExchange [Reuses operator id: 68] +(32) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#21] (33) BroadcastHashJoin [codegen id : 11] @@ -285,7 +284,7 @@ Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_da Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] Condition : (isnotnull(ws_bill_addr_sk#30) AND isnotnull(ws_item_sk#29)) -(47) ReusedExchange [Reuses operator id: 68] +(47) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#33] (48) BroadcastHashJoin [codegen id : 17] @@ -351,52 +350,48 @@ Functions [1]: [partial_sum(total_sales#16)] Aggregate Attributes [2]: [sum#41, isEmpty#42] Results [3]: [i_item_id#10, sum#43, isEmpty#44] -(61) Exchange -Input [3]: [i_item_id#10, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(62) HashAggregate [codegen id : 20] +(61) HashAggregate [codegen id : 19] Input [3]: [i_item_id#10, sum#43, isEmpty#44] Keys [1]: [i_item_id#10] Functions [1]: [sum(total_sales#16)] Aggregate Attributes [1]: [sum(total_sales#16)#45] Results [2]: [i_item_id#10, sum(total_sales#16)#45 AS total_sales#46] -(63) TakeOrderedAndProject +(62) TakeOrderedAndProject Input [2]: [i_item_id#10, total_sales#46] Arguments: 100, [total_sales#46 ASC NULLS FIRST], [i_item_id#10, total_sales#46] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * Project (67) - +- * Filter (66) - +- * ColumnarToRow (65) - +- Scan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (67) ++- * Project (66) + +- * Filter (65) + +- * ColumnarToRow (64) + +- Scan parquet spark_catalog.default.date_dim (63) -(64) Scan parquet spark_catalog.default.date_dim +(63) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#47, d_moy#48] 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 -(65) ColumnarToRow [codegen id : 1] +(64) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -(66) Filter [codegen id : 1] +(65) Filter [codegen id : 1] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 2001)) AND (d_moy#48 = 2)) AND isnotnull(d_date_sk#6)) -(67) Project [codegen id : 1] +(66) Project [codegen id : 1] Output [1]: [d_date_sk#6] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -(68) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 4177a855c93a9..e15e0921767f5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -1,105 +1,102 @@ TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (20) + WholeStageCodegen (19) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [i_item_id] - Filter [i_color] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (11) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (17) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [ca_address_sk] + Filter [ca_gmt_offset,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_id,i_item_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [i_item_id] + Filter [i_color] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_id,i_color] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #6 + WholeStageCodegen (11) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_addr_sk,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 + WholeStageCodegen (18) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #7 + WholeStageCodegen (17) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk,ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 61a1f4d927a4c..bcdaddbc10d87 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -1,67 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * Project (10) - : : +- * Filter (9) - : : +- * ColumnarToRow (8) - : : +- Scan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * Project (20) - : +- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet spark_catalog.default.item (17) - :- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * Filter (31) - : : : : +- * ColumnarToRow (30) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Filter (46) - : : : +- * ColumnarToRow (45) - : : : +- Scan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) +TakeOrderedAndProject (62) ++- * HashAggregate (61) + +- * HashAggregate (60) + +- Union (59) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * Project (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * Filter (16) + : : +- * ColumnarToRow (15) + : : +- Scan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * Project (20) + : +- * Filter (19) + : +- * ColumnarToRow (18) + : +- Scan parquet spark_catalog.default.item (17) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * Filter (31) + : : : : +- * ColumnarToRow (30) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * Filter (46) + : : : +- * ColumnarToRow (45) + : : : +- Scan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) (1) Scan parquet spark_catalog.default.store_sales @@ -79,7 +78,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) -(4) ReusedExchange [Reuses operator id: 68] +(4) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -213,7 +212,7 @@ Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_da Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) -(32) ReusedExchange [Reuses operator id: 68] +(32) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#21] (33) BroadcastHashJoin [codegen id : 11] @@ -285,7 +284,7 @@ Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_da Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] Condition : (isnotnull(ws_bill_addr_sk#30) AND isnotnull(ws_item_sk#29)) -(47) ReusedExchange [Reuses operator id: 68] +(47) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#33] (48) BroadcastHashJoin [codegen id : 17] @@ -351,52 +350,48 @@ Functions [1]: [partial_sum(total_sales#16)] Aggregate Attributes [2]: [sum#41, isEmpty#42] Results [3]: [i_item_id#10, sum#43, isEmpty#44] -(61) Exchange -Input [3]: [i_item_id#10, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(62) HashAggregate [codegen id : 20] +(61) HashAggregate [codegen id : 19] Input [3]: [i_item_id#10, sum#43, isEmpty#44] Keys [1]: [i_item_id#10] Functions [1]: [sum(total_sales#16)] Aggregate Attributes [1]: [sum(total_sales#16)#45] Results [2]: [i_item_id#10, sum(total_sales#16)#45 AS total_sales#46] -(63) TakeOrderedAndProject +(62) TakeOrderedAndProject Input [2]: [i_item_id#10, total_sales#46] Arguments: 100, [i_item_id#10 ASC NULLS FIRST, total_sales#46 ASC NULLS FIRST], [i_item_id#10, total_sales#46] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * Project (67) - +- * Filter (66) - +- * ColumnarToRow (65) - +- Scan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (67) ++- * Project (66) + +- * Filter (65) + +- * ColumnarToRow (64) + +- Scan parquet spark_catalog.default.date_dim (63) -(64) Scan parquet spark_catalog.default.date_dim +(63) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#47, d_moy#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 1] +(64) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -(66) Filter [codegen id : 1] +(65) Filter [codegen id : 1] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1998)) AND (d_moy#48 = 9)) AND isnotnull(d_date_sk#6)) -(67) Project [codegen id : 1] +(66) Project [codegen id : 1] Output [1]: [d_date_sk#6] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -(68) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt index 754cda4695efe..e396d2f9a147d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt @@ -1,105 +1,102 @@ TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (20) + WholeStageCodegen (19) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [i_item_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (11) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (17) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [ca_address_sk] + Filter [ca_gmt_offset,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_id,i_item_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [i_item_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_id,i_category] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #6 + WholeStageCodegen (11) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_addr_sk,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 + WholeStageCodegen (18) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #7 + WholeStageCodegen (17) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk,ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 61a1f4d927a4c..bcdaddbc10d87 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -1,67 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * Project (10) - : : +- * Filter (9) - : : +- * ColumnarToRow (8) - : : +- Scan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * Project (20) - : +- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet spark_catalog.default.item (17) - :- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * Filter (31) - : : : : +- * ColumnarToRow (30) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Filter (46) - : : : +- * ColumnarToRow (45) - : : : +- Scan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) +TakeOrderedAndProject (62) ++- * HashAggregate (61) + +- * HashAggregate (60) + +- Union (59) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * Project (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * Filter (16) + : : +- * ColumnarToRow (15) + : : +- Scan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * Project (20) + : +- * Filter (19) + : +- * ColumnarToRow (18) + : +- Scan parquet spark_catalog.default.item (17) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * Filter (31) + : : : : +- * ColumnarToRow (30) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * Filter (46) + : : : +- * ColumnarToRow (45) + : : : +- Scan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) (1) Scan parquet spark_catalog.default.store_sales @@ -79,7 +78,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) -(4) ReusedExchange [Reuses operator id: 68] +(4) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -213,7 +212,7 @@ Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_da Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) -(32) ReusedExchange [Reuses operator id: 68] +(32) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#21] (33) BroadcastHashJoin [codegen id : 11] @@ -285,7 +284,7 @@ Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_da Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] Condition : (isnotnull(ws_bill_addr_sk#30) AND isnotnull(ws_item_sk#29)) -(47) ReusedExchange [Reuses operator id: 68] +(47) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#33] (48) BroadcastHashJoin [codegen id : 17] @@ -351,52 +350,48 @@ Functions [1]: [partial_sum(total_sales#16)] Aggregate Attributes [2]: [sum#41, isEmpty#42] Results [3]: [i_item_id#10, sum#43, isEmpty#44] -(61) Exchange -Input [3]: [i_item_id#10, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(62) HashAggregate [codegen id : 20] +(61) HashAggregate [codegen id : 19] Input [3]: [i_item_id#10, sum#43, isEmpty#44] Keys [1]: [i_item_id#10] Functions [1]: [sum(total_sales#16)] Aggregate Attributes [1]: [sum(total_sales#16)#45] Results [2]: [i_item_id#10, sum(total_sales#16)#45 AS total_sales#46] -(63) TakeOrderedAndProject +(62) TakeOrderedAndProject Input [2]: [i_item_id#10, total_sales#46] Arguments: 100, [i_item_id#10 ASC NULLS FIRST, total_sales#46 ASC NULLS FIRST], [i_item_id#10, total_sales#46] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * Project (67) - +- * Filter (66) - +- * ColumnarToRow (65) - +- Scan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (67) ++- * Project (66) + +- * Filter (65) + +- * ColumnarToRow (64) + +- Scan parquet spark_catalog.default.date_dim (63) -(64) Scan parquet spark_catalog.default.date_dim +(63) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#47, d_moy#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 1] +(64) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -(66) Filter [codegen id : 1] +(65) Filter [codegen id : 1] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1998)) AND (d_moy#48 = 9)) AND isnotnull(d_date_sk#6)) -(67) Project [codegen id : 1] +(66) Project [codegen id : 1] Output [1]: [d_date_sk#6] Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -(68) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 754cda4695efe..e396d2f9a147d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -1,105 +1,102 @@ TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (20) + WholeStageCodegen (19) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [i_item_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (11) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (17) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [ca_address_sk] + Filter [ca_gmt_offset,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_id,i_item_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [i_item_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_id,i_category] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #6 + WholeStageCodegen (11) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_addr_sk,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 + WholeStageCodegen (18) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #7 + WholeStageCodegen (17) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk,ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt index 8b70f5c67ab31..609a6d729094f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt @@ -1,56 +1,55 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- Union (48) - :- * HashAggregate (29) - : +- Exchange (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * Project (17) - : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Project (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet spark_catalog.default.ship_mode (4) - : : : +- BroadcastExchange (15) - : : : +- * Project (14) - : : : +- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet spark_catalog.default.time_dim (11) - : : +- ReusedExchange (18) - : +- BroadcastExchange (24) - : +- * Filter (23) - : +- * ColumnarToRow (22) - : +- Scan parquet spark_catalog.default.warehouse (21) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- ReusedExchange (42) +TakeOrderedAndProject (51) ++- * HashAggregate (50) + +- * HashAggregate (49) + +- Union (48) + :- * HashAggregate (29) + : +- Exchange (28) + : +- * HashAggregate (27) + : +- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * Project (17) + : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.web_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * Project (7) + : : : : +- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet spark_catalog.default.ship_mode (4) + : : : +- BroadcastExchange (15) + : : : +- * Project (14) + : : : +- * Filter (13) + : : : +- * ColumnarToRow (12) + : : : +- Scan parquet spark_catalog.default.time_dim (11) + : : +- ReusedExchange (18) + : +- BroadcastExchange (24) + : +- * Filter (23) + : +- * ColumnarToRow (22) + : +- Scan parquet spark_catalog.default.warehouse (21) + +- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (30) + : : : +- ReusedExchange (33) + : : +- ReusedExchange (36) + : +- ReusedExchange (39) + +- ReusedExchange (42) (1) Scan parquet spark_catalog.default.web_sales @@ -132,7 +131,7 @@ Join condition: None Output [5]: [ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] Input [7]: [ws_sold_time_sk#1, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, t_time_sk#11] -(18) ReusedExchange [Reuses operator id: 56] +(18) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#13, d_year#14, d_moy#15] (19) BroadcastHashJoin [codegen id : 5] @@ -232,7 +231,7 @@ Join condition: None Output [5]: [cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] Input [7]: [cs_sold_time_sk#169, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, t_time_sk#177] -(39) ReusedExchange [Reuses operator id: 56] +(39) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#178, d_year#179, d_moy#180] (40) BroadcastHashJoin [codegen id : 11] @@ -285,47 +284,43 @@ Functions [36]: [partial_sum(jan_sales#145), partial_sum(feb_sales#146), partial Aggregate Attributes [72]: [sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405] Results [80]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477] -(50) Exchange -Input [80]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477] -Arguments: hashpartitioning(w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(51) HashAggregate [codegen id : 14] +(50) HashAggregate [codegen id : 13] Input [80]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477] Keys [8]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144] Functions [36]: [sum(jan_sales#145), sum(feb_sales#146), sum(mar_sales#147), sum(apr_sales#148), sum(may_sales#149), sum(jun_sales#150), sum(jul_sales#151), sum(aug_sales#152), sum(sep_sales#153), sum(oct_sales#154), sum(nov_sales#155), sum(dec_sales#156), sum((jan_sales#145 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((feb_sales#146 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((mar_sales#147 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((apr_sales#148 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((may_sales#149 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((jun_sales#150 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((jul_sales#151 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((aug_sales#152 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((sep_sales#153 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((oct_sales#154 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((nov_sales#155 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((dec_sales#156 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum(jan_net#157), sum(feb_net#158), sum(mar_net#159), sum(apr_net#160), sum(may_net#161), sum(jun_net#162), sum(jul_net#163), sum(aug_net#164), sum(sep_net#165), sum(oct_net#166), sum(nov_net#167), sum(dec_net#168)] Aggregate Attributes [36]: [sum(jan_sales#145)#478, sum(feb_sales#146)#479, sum(mar_sales#147)#480, sum(apr_sales#148)#481, sum(may_sales#149)#482, sum(jun_sales#150)#483, sum(jul_sales#151)#484, sum(aug_sales#152)#485, sum(sep_sales#153)#486, sum(oct_sales#154)#487, sum(nov_sales#155)#488, sum(dec_sales#156)#489, sum((jan_sales#145 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#490, sum((feb_sales#146 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#491, sum((mar_sales#147 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#492, sum((apr_sales#148 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#493, sum((may_sales#149 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#494, sum((jun_sales#150 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#495, sum((jul_sales#151 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#496, sum((aug_sales#152 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#497, sum((sep_sales#153 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#498, sum((oct_sales#154 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#499, sum((nov_sales#155 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#500, sum((dec_sales#156 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#501, sum(jan_net#157)#502, sum(feb_net#158)#503, sum(mar_net#159)#504, sum(apr_net#160)#505, sum(may_net#161)#506, sum(jun_net#162)#507, sum(jul_net#163)#508, sum(aug_net#164)#509, sum(sep_net#165)#510, sum(oct_net#166)#511, sum(nov_net#167)#512, sum(dec_net#168)#513] Results [44]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144, sum(jan_sales#145)#478 AS jan_sales#514, sum(feb_sales#146)#479 AS feb_sales#515, sum(mar_sales#147)#480 AS mar_sales#516, sum(apr_sales#148)#481 AS apr_sales#517, sum(may_sales#149)#482 AS may_sales#518, sum(jun_sales#150)#483 AS jun_sales#519, sum(jul_sales#151)#484 AS jul_sales#520, sum(aug_sales#152)#485 AS aug_sales#521, sum(sep_sales#153)#486 AS sep_sales#522, sum(oct_sales#154)#487 AS oct_sales#523, sum(nov_sales#155)#488 AS nov_sales#524, sum(dec_sales#156)#489 AS dec_sales#525, sum((jan_sales#145 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#490 AS jan_sales_per_sq_foot#526, sum((feb_sales#146 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#491 AS feb_sales_per_sq_foot#527, sum((mar_sales#147 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#492 AS mar_sales_per_sq_foot#528, sum((apr_sales#148 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#493 AS apr_sales_per_sq_foot#529, sum((may_sales#149 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#494 AS may_sales_per_sq_foot#530, sum((jun_sales#150 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#495 AS jun_sales_per_sq_foot#531, sum((jul_sales#151 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#496 AS jul_sales_per_sq_foot#532, sum((aug_sales#152 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#497 AS aug_sales_per_sq_foot#533, sum((sep_sales#153 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#498 AS sep_sales_per_sq_foot#534, sum((oct_sales#154 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#499 AS oct_sales_per_sq_foot#535, sum((nov_sales#155 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#500 AS nov_sales_per_sq_foot#536, sum((dec_sales#156 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#501 AS dec_sales_per_sq_foot#537, sum(jan_net#157)#502 AS jan_net#538, sum(feb_net#158)#503 AS feb_net#539, sum(mar_net#159)#504 AS mar_net#540, sum(apr_net#160)#505 AS apr_net#541, sum(may_net#161)#506 AS may_net#542, sum(jun_net#162)#507 AS jun_net#543, sum(jul_net#163)#508 AS jul_net#544, sum(aug_net#164)#509 AS aug_net#545, sum(sep_net#165)#510 AS sep_net#546, sum(oct_net#166)#511 AS oct_net#547, sum(nov_net#167)#512 AS nov_net#548, sum(dec_net#168)#513 AS dec_net#549] -(52) TakeOrderedAndProject +(51) TakeOrderedAndProject Input [44]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144, jan_sales#514, feb_sales#515, mar_sales#516, apr_sales#517, may_sales#518, jun_sales#519, jul_sales#520, aug_sales#521, sep_sales#522, oct_sales#523, nov_sales#524, dec_sales#525, jan_sales_per_sq_foot#526, feb_sales_per_sq_foot#527, mar_sales_per_sq_foot#528, apr_sales_per_sq_foot#529, may_sales_per_sq_foot#530, jun_sales_per_sq_foot#531, jul_sales_per_sq_foot#532, aug_sales_per_sq_foot#533, sep_sales_per_sq_foot#534, oct_sales_per_sq_foot#535, nov_sales_per_sq_foot#536, dec_sales_per_sq_foot#537, jan_net#538, feb_net#539, mar_net#540, apr_net#541, may_net#542, jun_net#543, jul_net#544, aug_net#545, sep_net#546, oct_net#547, nov_net#548, dec_net#549] Arguments: 100, [w_warehouse_name#17 ASC NULLS FIRST], [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144, jan_sales#514, feb_sales#515, mar_sales#516, apr_sales#517, may_sales#518, jun_sales#519, jul_sales#520, aug_sales#521, sep_sales#522, oct_sales#523, nov_sales#524, dec_sales#525, jan_sales_per_sq_foot#526, feb_sales_per_sq_foot#527, mar_sales_per_sq_foot#528, apr_sales_per_sq_foot#529, may_sales_per_sq_foot#530, jun_sales_per_sq_foot#531, jul_sales_per_sq_foot#532, aug_sales_per_sq_foot#533, sep_sales_per_sq_foot#534, oct_sales_per_sq_foot#535, nov_sales_per_sq_foot#536, dec_sales_per_sq_foot#537, jan_net#538, feb_net#539, mar_net#540, apr_net#541, may_net#542, jun_net#543, jul_net#544, aug_net#545, sep_net#546, oct_net#547, nov_net#548, dec_net#549] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (56) -+- * Filter (55) - +- * ColumnarToRow (54) - +- Scan parquet spark_catalog.default.date_dim (53) +BroadcastExchange (55) ++- * Filter (54) + +- * ColumnarToRow (53) + +- Scan parquet spark_catalog.default.date_dim (52) -(53) Scan parquet spark_catalog.default.date_dim +(52) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -(55) Filter [codegen id : 1] +(54) Filter [codegen id : 1] Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) -(56) BroadcastExchange +(55) BroadcastExchange Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#175 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt index 53e8d4a171ba7..f2d257330feb1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt @@ -1,86 +1,83 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - WholeStageCodegen (14) + WholeStageCodegen (13) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - WholeStageCodegen (13) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - WholeStageCodegen (5) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - Project [ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,d_year,d_moy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Filter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [sm_ship_mode_sk] - Filter [sm_carrier,sm_ship_mode_sk] + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #1 + WholeStageCodegen (5) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + Project [ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,d_year,d_moy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [ws_sold_time_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Filter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Project [t_time_sk] - Filter [t_time,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [sm_ship_mode_sk] + Filter [sm_carrier,sm_ship_mode_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Project [t_time_sk] + Filter [t_time,t_time_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #2 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + Filter [w_warehouse_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [w_warehouse_sk] + Scan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + WholeStageCodegen (12) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #6 + WholeStageCodegen (11) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + Project [cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] + BroadcastHashJoin [cs_sold_time_sk,t_time_sk] + Project [cs_sold_time_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] + BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + Filter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - WholeStageCodegen (12) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - WholeStageCodegen (11) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Project [cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] - BroadcastHashJoin [cs_sold_time_sk,t_time_sk] - Project [cs_sold_time_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Filter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [sm_ship_mode_sk] #4 - InputAdapter - ReusedExchange [t_time_sk] #5 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #6 + Scan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [sm_ship_mode_sk] #3 + InputAdapter + ReusedExchange [t_time_sk] #4 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #2 + InputAdapter + ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index 09c68efe6f11d..02a57a28f9ab0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -1,56 +1,55 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- Union (48) - :- * HashAggregate (29) - : +- Exchange (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet spark_catalog.default.warehouse (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (17) - : : +- * Project (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet spark_catalog.default.time_dim (13) - : +- BroadcastExchange (24) - : +- * Project (23) - : +- * Filter (22) - : +- * ColumnarToRow (21) - : +- Scan parquet spark_catalog.default.ship_mode (20) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- ReusedExchange (42) +TakeOrderedAndProject (51) ++- * HashAggregate (50) + +- * HashAggregate (49) + +- Union (48) + :- * HashAggregate (29) + : +- Exchange (28) + : +- * HashAggregate (27) + : +- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.web_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet spark_catalog.default.warehouse (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (17) + : : +- * Project (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet spark_catalog.default.time_dim (13) + : +- BroadcastExchange (24) + : +- * Project (23) + : +- * Filter (22) + : +- * ColumnarToRow (21) + : +- Scan parquet spark_catalog.default.ship_mode (20) + +- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (30) + : : : +- ReusedExchange (33) + : : +- ReusedExchange (36) + : +- ReusedExchange (39) + +- ReusedExchange (42) (1) Scan parquet spark_catalog.default.web_sales @@ -96,7 +95,7 @@ Join condition: None Output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -(10) ReusedExchange [Reuses operator id: 56] +(10) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#16, d_year#17, d_moy#18] (11) BroadcastHashJoin [codegen id : 5] @@ -219,7 +218,7 @@ Join condition: None Output [12]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#177, w_warehouse_sq_ft#178, w_city#179, w_county#180, w_state#181, w_country#182] Input [14]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_sk#176, w_warehouse_name#177, w_warehouse_sq_ft#178, w_city#179, w_county#180, w_state#181, w_country#182] -(36) ReusedExchange [Reuses operator id: 56] +(36) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#183, d_year#184, d_moy#185] (37) BroadcastHashJoin [codegen id : 11] @@ -285,47 +284,43 @@ Functions [36]: [partial_sum(jan_sales#145), partial_sum(feb_sales#146), partial Aggregate Attributes [72]: [sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405] Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477] -(50) Exchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(51) HashAggregate [codegen id : 14] +(50) HashAggregate [codegen id : 13] Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144] Functions [36]: [sum(jan_sales#145), sum(feb_sales#146), sum(mar_sales#147), sum(apr_sales#148), sum(may_sales#149), sum(jun_sales#150), sum(jul_sales#151), sum(aug_sales#152), sum(sep_sales#153), sum(oct_sales#154), sum(nov_sales#155), sum(dec_sales#156), sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#157), sum(feb_net#158), sum(mar_net#159), sum(apr_net#160), sum(may_net#161), sum(jun_net#162), sum(jul_net#163), sum(aug_net#164), sum(sep_net#165), sum(oct_net#166), sum(nov_net#167), sum(dec_net#168)] Aggregate Attributes [36]: [sum(jan_sales#145)#478, sum(feb_sales#146)#479, sum(mar_sales#147)#480, sum(apr_sales#148)#481, sum(may_sales#149)#482, sum(jun_sales#150)#483, sum(jul_sales#151)#484, sum(aug_sales#152)#485, sum(sep_sales#153)#486, sum(oct_sales#154)#487, sum(nov_sales#155)#488, sum(dec_sales#156)#489, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#490, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501, sum(jan_net#157)#502, sum(feb_net#158)#503, sum(mar_net#159)#504, sum(apr_net#160)#505, sum(may_net#161)#506, sum(jun_net#162)#507, sum(jul_net#163)#508, sum(aug_net#164)#509, sum(sep_net#165)#510, sum(oct_net#166)#511, sum(nov_net#167)#512, sum(dec_net#168)#513] Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum(jan_sales#145)#478 AS jan_sales#514, sum(feb_sales#146)#479 AS feb_sales#515, sum(mar_sales#147)#480 AS mar_sales#516, sum(apr_sales#148)#481 AS apr_sales#517, sum(may_sales#149)#482 AS may_sales#518, sum(jun_sales#150)#483 AS jun_sales#519, sum(jul_sales#151)#484 AS jul_sales#520, sum(aug_sales#152)#485 AS aug_sales#521, sum(sep_sales#153)#486 AS sep_sales#522, sum(oct_sales#154)#487 AS oct_sales#523, sum(nov_sales#155)#488 AS nov_sales#524, sum(dec_sales#156)#489 AS dec_sales#525, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#490 AS jan_sales_per_sq_foot#526, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491 AS feb_sales_per_sq_foot#527, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492 AS mar_sales_per_sq_foot#528, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493 AS apr_sales_per_sq_foot#529, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494 AS may_sales_per_sq_foot#530, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495 AS jun_sales_per_sq_foot#531, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496 AS jul_sales_per_sq_foot#532, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497 AS aug_sales_per_sq_foot#533, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498 AS sep_sales_per_sq_foot#534, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499 AS oct_sales_per_sq_foot#535, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500 AS nov_sales_per_sq_foot#536, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501 AS dec_sales_per_sq_foot#537, sum(jan_net#157)#502 AS jan_net#538, sum(feb_net#158)#503 AS feb_net#539, sum(mar_net#159)#504 AS mar_net#540, sum(apr_net#160)#505 AS apr_net#541, sum(may_net#161)#506 AS may_net#542, sum(jun_net#162)#507 AS jun_net#543, sum(jul_net#163)#508 AS jul_net#544, sum(aug_net#164)#509 AS aug_net#545, sum(sep_net#165)#510 AS sep_net#546, sum(oct_net#166)#511 AS oct_net#547, sum(nov_net#167)#512 AS nov_net#548, sum(dec_net#168)#513 AS dec_net#549] -(52) TakeOrderedAndProject +(51) TakeOrderedAndProject Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#514, feb_sales#515, mar_sales#516, apr_sales#517, may_sales#518, jun_sales#519, jul_sales#520, aug_sales#521, sep_sales#522, oct_sales#523, nov_sales#524, dec_sales#525, jan_sales_per_sq_foot#526, feb_sales_per_sq_foot#527, mar_sales_per_sq_foot#528, apr_sales_per_sq_foot#529, may_sales_per_sq_foot#530, jun_sales_per_sq_foot#531, jul_sales_per_sq_foot#532, aug_sales_per_sq_foot#533, sep_sales_per_sq_foot#534, oct_sales_per_sq_foot#535, nov_sales_per_sq_foot#536, dec_sales_per_sq_foot#537, jan_net#538, feb_net#539, mar_net#540, apr_net#541, may_net#542, jun_net#543, jul_net#544, aug_net#545, sep_net#546, oct_net#547, nov_net#548, dec_net#549] Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#514, feb_sales#515, mar_sales#516, apr_sales#517, may_sales#518, jun_sales#519, jul_sales#520, aug_sales#521, sep_sales#522, oct_sales#523, nov_sales#524, dec_sales#525, jan_sales_per_sq_foot#526, feb_sales_per_sq_foot#527, mar_sales_per_sq_foot#528, apr_sales_per_sq_foot#529, may_sales_per_sq_foot#530, jun_sales_per_sq_foot#531, jul_sales_per_sq_foot#532, aug_sales_per_sq_foot#533, sep_sales_per_sq_foot#534, oct_sales_per_sq_foot#535, nov_sales_per_sq_foot#536, dec_sales_per_sq_foot#537, jan_net#538, feb_net#539, mar_net#540, apr_net#541, may_net#542, jun_net#543, jul_net#544, aug_net#545, sep_net#546, oct_net#547, nov_net#548, dec_net#549] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (56) -+- * Filter (55) - +- * ColumnarToRow (54) - +- Scan parquet spark_catalog.default.date_dim (53) +BroadcastExchange (55) ++- * Filter (54) + +- * ColumnarToRow (53) + +- Scan parquet spark_catalog.default.date_dim (52) -(53) Scan parquet spark_catalog.default.date_dim +(52) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#16, d_year#17, d_moy#18] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -(55) Filter [codegen id : 1] +(54) Filter [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(56) BroadcastExchange +(55) BroadcastExchange Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#175 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt index e0cb9e9cf6059..abcf58dcae90f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt @@ -1,86 +1,83 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - WholeStageCodegen (14) + WholeStageCodegen (13) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - WholeStageCodegen (13) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - WholeStageCodegen (5) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - Filter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #1 + WholeStageCodegen (5) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Project [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + Filter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [t_time_sk] - Filter [t_time,t_time_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Project [sm_ship_mode_sk] - Filter [sm_carrier,sm_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - WholeStageCodegen (12) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - WholeStageCodegen (11) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_time_sk,t_time_sk] - Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Filter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - ReusedExchange [t_time_sk] #5 - InputAdapter - ReusedExchange [sm_ship_mode_sk] #6 + Scan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #2 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + Project [t_time_sk] + Filter [t_time,t_time_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + Project [sm_ship_mode_sk] + Filter [sm_carrier,sm_ship_mode_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + WholeStageCodegen (12) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #6 + WholeStageCodegen (11) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + Project [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_sold_time_sk,t_time_sk] + Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + Filter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #3 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #2 + InputAdapter + ReusedExchange [t_time_sk] #4 + InputAdapter + ReusedExchange [sm_ship_mode_sk] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt index e9e4b66c17366..73858da641308 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt @@ -1,135 +1,174 @@ == Physical Plan == -TakeOrderedAndProject (131) -+- * HashAggregate (130) - +- Exchange (129) - +- * HashAggregate (128) - +- Union (127) - :- * HashAggregate (106) - : +- Exchange (105) - : +- * HashAggregate (104) - : +- Union (103) - : :- * Filter (72) - : : +- * HashAggregate (71) - : : +- Exchange (70) - : : +- * HashAggregate (69) - : : +- * Project (68) - : : +- * BroadcastHashJoin Inner BuildRight (67) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (56) - : : : : +- * Project (55) - : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (53) - : : : : +- * SortMergeJoin LeftSemi (52) - : : : : :- * Sort (40) - : : : : : +- Exchange (39) - : : : : : +- * HashAggregate (38) - : : : : : +- Exchange (37) - : : : : : +- * HashAggregate (36) - : : : : : +- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (12) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : : +- ReusedExchange (10) - : : : : : +- BroadcastExchange (33) - : : : : : +- * SortMergeJoin LeftSemi (32) - : : : : : :- * Sort (17) - : : : : : : +- Exchange (16) - : : : : : : +- * Filter (15) - : : : : : : +- * ColumnarToRow (14) - : : : : : : +- Scan parquet spark_catalog.default.item (13) - : : : : : +- * Sort (31) - : : : : : +- Exchange (30) - : : : : : +- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Project (23) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : : :- * Filter (20) - : : : : : : : +- * ColumnarToRow (19) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : : : : : +- ReusedExchange (21) - : : : : : +- BroadcastExchange (27) - : : : : : +- * Filter (26) - : : : : : +- * ColumnarToRow (25) - : : : : : +- Scan parquet spark_catalog.default.item (24) - : : : : +- * Sort (51) - : : : : +- Exchange (50) - : : : : +- * Project (49) - : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : :- * Project (46) - : : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : : :- * Filter (43) - : : : : : : +- * ColumnarToRow (42) - : : : : : : +- Scan parquet spark_catalog.default.web_sales (41) - : : : : : +- ReusedExchange (44) - : : : : +- ReusedExchange (47) - : : : +- ReusedExchange (58) - : : +- BroadcastExchange (66) - : : +- * BroadcastHashJoin LeftSemi BuildRight (65) - : : :- * Filter (63) - : : : +- * ColumnarToRow (62) - : : : +- Scan parquet spark_catalog.default.item (61) - : : +- ReusedExchange (64) - : :- * Filter (87) - : : +- * HashAggregate (86) - : : +- Exchange (85) - : : +- * HashAggregate (84) - : : +- * Project (83) - : : +- * BroadcastHashJoin Inner BuildRight (82) - : : :- * Project (80) - : : : +- * BroadcastHashJoin Inner BuildRight (79) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (77) - : : : : :- * Filter (75) - : : : : : +- * ColumnarToRow (74) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (73) - : : : : +- ReusedExchange (76) - : : : +- ReusedExchange (78) - : : +- ReusedExchange (81) - : +- * Filter (102) - : +- * HashAggregate (101) - : +- Exchange (100) - : +- * HashAggregate (99) - : +- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * BroadcastHashJoin LeftSemi BuildRight (92) - : : : :- * Filter (90) - : : : : +- * ColumnarToRow (89) - : : : : +- Scan parquet spark_catalog.default.web_sales (88) - : : : +- ReusedExchange (91) - : : +- ReusedExchange (93) - : +- ReusedExchange (96) - :- * HashAggregate (111) - : +- Exchange (110) - : +- * HashAggregate (109) - : +- * HashAggregate (108) - : +- ReusedExchange (107) - :- * HashAggregate (116) - : +- Exchange (115) - : +- * HashAggregate (114) - : +- * HashAggregate (113) - : +- ReusedExchange (112) - :- * HashAggregate (121) - : +- Exchange (120) - : +- * HashAggregate (119) - : +- * HashAggregate (118) - : +- ReusedExchange (117) - +- * HashAggregate (126) - +- Exchange (125) - +- * HashAggregate (124) - +- * HashAggregate (123) - +- ReusedExchange (122) +TakeOrderedAndProject (170) ++- * HashAggregate (169) + +- Exchange (168) + +- * HashAggregate (167) + +- Union (166) + :- * HashAggregate (105) + : +- * HashAggregate (104) + : +- Union (103) + : :- * Filter (72) + : : +- * HashAggregate (71) + : : +- Exchange (70) + : : +- * HashAggregate (69) + : : +- * Project (68) + : : +- * BroadcastHashJoin Inner BuildRight (67) + : : :- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (57) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (56) + : : : : +- * Project (55) + : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet spark_catalog.default.item (4) + : : : : +- BroadcastExchange (53) + : : : : +- * SortMergeJoin LeftSemi (52) + : : : : :- * Sort (40) + : : : : : +- Exchange (39) + : : : : : +- * HashAggregate (38) + : : : : : +- Exchange (37) + : : : : : +- * HashAggregate (36) + : : : : : +- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (12) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : : : +- ReusedExchange (10) + : : : : : +- BroadcastExchange (33) + : : : : : +- * SortMergeJoin LeftSemi (32) + : : : : : :- * Sort (17) + : : : : : : +- Exchange (16) + : : : : : : +- * Filter (15) + : : : : : : +- * ColumnarToRow (14) + : : : : : : +- Scan parquet spark_catalog.default.item (13) + : : : : : +- * Sort (31) + : : : : : +- Exchange (30) + : : : : : +- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Project (23) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : : : :- * Filter (20) + : : : : : : : +- * ColumnarToRow (19) + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (18) + : : : : : : +- ReusedExchange (21) + : : : : : +- BroadcastExchange (27) + : : : : : +- * Filter (26) + : : : : : +- * ColumnarToRow (25) + : : : : : +- Scan parquet spark_catalog.default.item (24) + : : : : +- * Sort (51) + : : : : +- Exchange (50) + : : : : +- * Project (49) + : : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : : :- * Project (46) + : : : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : : : :- * Filter (43) + : : : : : : +- * ColumnarToRow (42) + : : : : : : +- Scan parquet spark_catalog.default.web_sales (41) + : : : : : +- ReusedExchange (44) + : : : : +- ReusedExchange (47) + : : : +- ReusedExchange (58) + : : +- BroadcastExchange (66) + : : +- * BroadcastHashJoin LeftSemi BuildRight (65) + : : :- * Filter (63) + : : : +- * ColumnarToRow (62) + : : : +- Scan parquet spark_catalog.default.item (61) + : : +- ReusedExchange (64) + : :- * Filter (87) + : : +- * HashAggregate (86) + : : +- Exchange (85) + : : +- * HashAggregate (84) + : : +- * Project (83) + : : +- * BroadcastHashJoin Inner BuildRight (82) + : : :- * Project (80) + : : : +- * BroadcastHashJoin Inner BuildRight (79) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (77) + : : : : :- * Filter (75) + : : : : : +- * ColumnarToRow (74) + : : : : : +- Scan parquet spark_catalog.default.catalog_sales (73) + : : : : +- ReusedExchange (76) + : : : +- ReusedExchange (78) + : : +- ReusedExchange (81) + : +- * Filter (102) + : +- * HashAggregate (101) + : +- Exchange (100) + : +- * HashAggregate (99) + : +- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- * Project (95) + : : +- * BroadcastHashJoin Inner BuildRight (94) + : : :- * BroadcastHashJoin LeftSemi BuildRight (92) + : : : :- * Filter (90) + : : : : +- * ColumnarToRow (89) + : : : : +- Scan parquet spark_catalog.default.web_sales (88) + : : : +- ReusedExchange (91) + : : +- ReusedExchange (93) + : +- ReusedExchange (96) + :- * HashAggregate (120) + : +- Exchange (119) + : +- * HashAggregate (118) + : +- * HashAggregate (117) + : +- * HashAggregate (116) + : +- Union (115) + : :- * Filter (108) + : : +- * HashAggregate (107) + : : +- ReusedExchange (106) + : :- * Filter (111) + : : +- * HashAggregate (110) + : : +- ReusedExchange (109) + : +- * Filter (114) + : +- * HashAggregate (113) + : +- ReusedExchange (112) + :- * HashAggregate (135) + : +- Exchange (134) + : +- * HashAggregate (133) + : +- * HashAggregate (132) + : +- * HashAggregate (131) + : +- Union (130) + : :- * Filter (123) + : : +- * HashAggregate (122) + : : +- ReusedExchange (121) + : :- * Filter (126) + : : +- * HashAggregate (125) + : : +- ReusedExchange (124) + : +- * Filter (129) + : +- * HashAggregate (128) + : +- ReusedExchange (127) + :- * HashAggregate (150) + : +- Exchange (149) + : +- * HashAggregate (148) + : +- * HashAggregate (147) + : +- * HashAggregate (146) + : +- Union (145) + : :- * Filter (138) + : : +- * HashAggregate (137) + : : +- ReusedExchange (136) + : :- * Filter (141) + : : +- * HashAggregate (140) + : : +- ReusedExchange (139) + : +- * Filter (144) + : +- * HashAggregate (143) + : +- ReusedExchange (142) + +- * HashAggregate (165) + +- Exchange (164) + +- * HashAggregate (163) + +- * HashAggregate (162) + +- * HashAggregate (161) + +- Union (160) + :- * Filter (153) + : +- * HashAggregate (152) + : +- ReusedExchange (151) + :- * Filter (156) + : +- * HashAggregate (155) + : +- ReusedExchange (154) + +- * Filter (159) + +- * HashAggregate (158) + +- ReusedExchange (157) (1) Scan parquet spark_catalog.default.store_sales @@ -176,7 +215,7 @@ Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(10) ReusedExchange [Reuses operator id: 165] +(10) ReusedExchange [Reuses operator id: 204] Output [1]: [d_date_sk#13] (11) BroadcastHashJoin [codegen id : 9] @@ -226,7 +265,7 @@ Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Condition : isnotnull(cs_item_sk#18) -(21) ReusedExchange [Reuses operator id: 165] +(21) ReusedExchange [Reuses operator id: 204] Output [1]: [d_date_sk#20] (22) BroadcastHashJoin [codegen id : 6] @@ -336,7 +375,7 @@ Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) -(44) ReusedExchange [Reuses operator id: 165] +(44) ReusedExchange [Reuses operator id: 204] Output [1]: [d_date_sk#30] (45) BroadcastHashJoin [codegen id : 14] @@ -400,7 +439,7 @@ Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(58) ReusedExchange [Reuses operator id: 160] +(58) ReusedExchange [Reuses operator id: 199] Output [1]: [d_date_sk#36] (59) BroadcastHashJoin [codegen id : 37] @@ -496,7 +535,7 @@ Right keys [1]: [ss_item_sk#57] Join type: LeftSemi Join condition: None -(78) ReusedExchange [Reuses operator id: 160] +(78) ReusedExchange [Reuses operator id: 199] Output [1]: [d_date_sk#58] (79) BroadcastHashJoin [codegen id : 75] @@ -568,7 +607,7 @@ Right keys [1]: [ss_item_sk#78] Join type: LeftSemi Join condition: None -(93) ReusedExchange [Reuses operator id: 160] +(93) ReusedExchange [Reuses operator id: 199] Output [1]: [d_date_sk#79] (94) BroadcastHashJoin [codegen id : 113] @@ -625,359 +664,547 @@ Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] Aggregate Attributes [3]: [sum#95, isEmpty#96, sum#97] Results [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#98, isEmpty#99, sum#100] -(105) Exchange -Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#98, isEmpty#99, sum#100] -Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=15] - -(106) HashAggregate [codegen id : 116] +(105) HashAggregate [codegen id : 115] Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#98, isEmpty#99, sum#100] Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#101, sum(number_sales#51)#102] Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum(sales#50)#101 AS sum_sales#103, sum(number_sales#51)#102 AS number_sales#104] -(107) ReusedExchange [Reuses operator id: 105] -Output [7]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum#109, isEmpty#110, sum#111] - -(108) HashAggregate [codegen id : 232] -Input [7]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum#109, isEmpty#110, sum#111] -Keys [4]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108] -Functions [2]: [sum(sales#112), sum(number_sales#113)] -Aggregate Attributes [2]: [sum(sales#112)#101, sum(number_sales#113)#102] -Results [5]: [channel#105, i_brand_id#106, i_class_id#107, sum(sales#112)#101 AS sum_sales#114, sum(number_sales#113)#102 AS number_sales#115] - -(109) HashAggregate [codegen id : 232] -Input [5]: [channel#105, i_brand_id#106, i_class_id#107, sum_sales#114, number_sales#115] -Keys [3]: [channel#105, i_brand_id#106, i_class_id#107] -Functions [2]: [partial_sum(sum_sales#114), partial_sum(number_sales#115)] -Aggregate Attributes [3]: [sum#116, isEmpty#117, sum#118] -Results [6]: [channel#105, i_brand_id#106, i_class_id#107, sum#119, isEmpty#120, sum#121] - -(110) Exchange -Input [6]: [channel#105, i_brand_id#106, i_class_id#107, sum#119, isEmpty#120, sum#121] -Arguments: hashpartitioning(channel#105, i_brand_id#106, i_class_id#107, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(111) HashAggregate [codegen id : 233] -Input [6]: [channel#105, i_brand_id#106, i_class_id#107, sum#119, isEmpty#120, sum#121] -Keys [3]: [channel#105, i_brand_id#106, i_class_id#107] -Functions [2]: [sum(sum_sales#114), sum(number_sales#115)] -Aggregate Attributes [2]: [sum(sum_sales#114)#122, sum(number_sales#115)#123] -Results [6]: [channel#105, i_brand_id#106, i_class_id#107, null AS i_category_id#124, sum(sum_sales#114)#122 AS sum(sum_sales)#125, sum(number_sales#115)#123 AS sum(number_sales)#126] - -(112) ReusedExchange [Reuses operator id: 105] -Output [7]: [channel#127, i_brand_id#128, i_class_id#129, i_category_id#130, sum#131, isEmpty#132, sum#133] - -(113) HashAggregate [codegen id : 349] -Input [7]: [channel#127, i_brand_id#128, i_class_id#129, i_category_id#130, sum#131, isEmpty#132, sum#133] -Keys [4]: [channel#127, i_brand_id#128, i_class_id#129, i_category_id#130] -Functions [2]: [sum(sales#134), sum(number_sales#135)] -Aggregate Attributes [2]: [sum(sales#134)#101, sum(number_sales#135)#102] -Results [4]: [channel#127, i_brand_id#128, sum(sales#134)#101 AS sum_sales#136, sum(number_sales#135)#102 AS number_sales#137] - -(114) HashAggregate [codegen id : 349] -Input [4]: [channel#127, i_brand_id#128, sum_sales#136, number_sales#137] -Keys [2]: [channel#127, i_brand_id#128] -Functions [2]: [partial_sum(sum_sales#136), partial_sum(number_sales#137)] -Aggregate Attributes [3]: [sum#138, isEmpty#139, sum#140] -Results [5]: [channel#127, i_brand_id#128, sum#141, isEmpty#142, sum#143] - -(115) Exchange -Input [5]: [channel#127, i_brand_id#128, sum#141, isEmpty#142, sum#143] -Arguments: hashpartitioning(channel#127, i_brand_id#128, 5), ENSURE_REQUIREMENTS, [plan_id=17] - -(116) HashAggregate [codegen id : 350] -Input [5]: [channel#127, i_brand_id#128, sum#141, isEmpty#142, sum#143] -Keys [2]: [channel#127, i_brand_id#128] -Functions [2]: [sum(sum_sales#136), sum(number_sales#137)] -Aggregate Attributes [2]: [sum(sum_sales#136)#144, sum(number_sales#137)#145] -Results [6]: [channel#127, i_brand_id#128, null AS i_class_id#146, null AS i_category_id#147, sum(sum_sales#136)#144 AS sum(sum_sales)#148, sum(number_sales#137)#145 AS sum(number_sales)#149] - -(117) ReusedExchange [Reuses operator id: 105] -Output [7]: [channel#150, i_brand_id#151, i_class_id#152, i_category_id#153, sum#154, isEmpty#155, sum#156] - -(118) HashAggregate [codegen id : 466] -Input [7]: [channel#150, i_brand_id#151, i_class_id#152, i_category_id#153, sum#154, isEmpty#155, sum#156] -Keys [4]: [channel#150, i_brand_id#151, i_class_id#152, i_category_id#153] -Functions [2]: [sum(sales#157), sum(number_sales#158)] -Aggregate Attributes [2]: [sum(sales#157)#101, sum(number_sales#158)#102] -Results [3]: [channel#150, sum(sales#157)#101 AS sum_sales#159, sum(number_sales#158)#102 AS number_sales#160] - -(119) HashAggregate [codegen id : 466] -Input [3]: [channel#150, sum_sales#159, number_sales#160] -Keys [1]: [channel#150] -Functions [2]: [partial_sum(sum_sales#159), partial_sum(number_sales#160)] -Aggregate Attributes [3]: [sum#161, isEmpty#162, sum#163] -Results [4]: [channel#150, sum#164, isEmpty#165, sum#166] - -(120) Exchange -Input [4]: [channel#150, sum#164, isEmpty#165, sum#166] -Arguments: hashpartitioning(channel#150, 5), ENSURE_REQUIREMENTS, [plan_id=18] - -(121) HashAggregate [codegen id : 467] -Input [4]: [channel#150, sum#164, isEmpty#165, sum#166] -Keys [1]: [channel#150] -Functions [2]: [sum(sum_sales#159), sum(number_sales#160)] -Aggregate Attributes [2]: [sum(sum_sales#159)#167, sum(number_sales#160)#168] -Results [6]: [channel#150, null AS i_brand_id#169, null AS i_class_id#170, null AS i_category_id#171, sum(sum_sales#159)#167 AS sum(sum_sales)#172, sum(number_sales#160)#168 AS sum(number_sales)#173] - -(122) ReusedExchange [Reuses operator id: 105] -Output [7]: [channel#174, i_brand_id#175, i_class_id#176, i_category_id#177, sum#178, isEmpty#179, sum#180] - -(123) HashAggregate [codegen id : 583] -Input [7]: [channel#174, i_brand_id#175, i_class_id#176, i_category_id#177, sum#178, isEmpty#179, sum#180] -Keys [4]: [channel#174, i_brand_id#175, i_class_id#176, i_category_id#177] -Functions [2]: [sum(sales#181), sum(number_sales#182)] -Aggregate Attributes [2]: [sum(sales#181)#101, sum(number_sales#182)#102] -Results [2]: [sum(sales#181)#101 AS sum_sales#183, sum(number_sales#182)#102 AS number_sales#184] - -(124) HashAggregate [codegen id : 583] -Input [2]: [sum_sales#183, number_sales#184] +(106) ReusedExchange [Reuses operator id: 70] +Output [6]: [i_brand_id#105, i_class_id#106, i_category_id#107, sum#108, isEmpty#109, count#46] + +(107) HashAggregate [codegen id : 153] +Input [6]: [i_brand_id#105, i_class_id#106, i_category_id#107, sum#108, isEmpty#109, count#46] +Keys [3]: [i_brand_id#105, i_class_id#106, i_category_id#107] +Functions [2]: [sum((cast(ss_quantity#110 as decimal(10,0)) * ss_list_price#111)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#110 as decimal(10,0)) * ss_list_price#111))#47, count(1)#48] +Results [6]: [store AS channel#112, i_brand_id#105, i_class_id#106, i_category_id#107, sum((cast(ss_quantity#110 as decimal(10,0)) * ss_list_price#111))#47 AS sales#113, count(1)#48 AS number_sales#114] + +(108) Filter [codegen id : 153] +Input [6]: [channel#112, i_brand_id#105, i_class_id#106, i_category_id#107, sales#113, number_sales#114] +Condition : (isnotnull(sales#113) AND (cast(sales#113 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#12] as decimal(32,6)))) + +(109) ReusedExchange [Reuses operator id: 85] +Output [6]: [i_brand_id#115, i_class_id#116, i_category_id#117, sum#118, isEmpty#119, count#68] + +(110) HashAggregate [codegen id : 191] +Input [6]: [i_brand_id#115, i_class_id#116, i_category_id#117, sum#118, isEmpty#119, count#68] +Keys [3]: [i_brand_id#115, i_class_id#116, i_category_id#117] +Functions [2]: [sum((cast(cs_quantity#120 as decimal(10,0)) * cs_list_price#121)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#120 as decimal(10,0)) * cs_list_price#121))#69, count(1)#70] +Results [6]: [catalog AS channel#122, i_brand_id#115, i_class_id#116, i_category_id#117, sum((cast(cs_quantity#120 as decimal(10,0)) * cs_list_price#121))#69 AS sales#123, count(1)#70 AS number_sales#124] + +(111) Filter [codegen id : 191] +Input [6]: [channel#122, i_brand_id#115, i_class_id#116, i_category_id#117, sales#123, number_sales#124] +Condition : (isnotnull(sales#123) AND (cast(sales#123 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#12] as decimal(32,6)))) + +(112) ReusedExchange [Reuses operator id: 100] +Output [6]: [i_brand_id#125, i_class_id#126, i_category_id#127, sum#128, isEmpty#129, count#89] + +(113) HashAggregate [codegen id : 229] +Input [6]: [i_brand_id#125, i_class_id#126, i_category_id#127, sum#128, isEmpty#129, count#89] +Keys [3]: [i_brand_id#125, i_class_id#126, i_category_id#127] +Functions [2]: [sum((cast(ws_quantity#130 as decimal(10,0)) * ws_list_price#131)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#130 as decimal(10,0)) * ws_list_price#131))#90, count(1)#91] +Results [6]: [web AS channel#132, i_brand_id#125, i_class_id#126, i_category_id#127, sum((cast(ws_quantity#130 as decimal(10,0)) * ws_list_price#131))#90 AS sales#133, count(1)#91 AS number_sales#134] + +(114) Filter [codegen id : 229] +Input [6]: [channel#132, i_brand_id#125, i_class_id#126, i_category_id#127, sales#133, number_sales#134] +Condition : (isnotnull(sales#133) AND (cast(sales#133 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#12] as decimal(32,6)))) + +(115) Union + +(116) HashAggregate [codegen id : 230] +Input [6]: [channel#112, i_brand_id#105, i_class_id#106, i_category_id#107, sales#113, number_sales#114] +Keys [4]: [channel#112, i_brand_id#105, i_class_id#106, i_category_id#107] +Functions [2]: [partial_sum(sales#113), partial_sum(number_sales#114)] +Aggregate Attributes [3]: [sum#135, isEmpty#136, sum#137] +Results [7]: [channel#112, i_brand_id#105, i_class_id#106, i_category_id#107, sum#138, isEmpty#139, sum#140] + +(117) HashAggregate [codegen id : 230] +Input [7]: [channel#112, i_brand_id#105, i_class_id#106, i_category_id#107, sum#138, isEmpty#139, sum#140] +Keys [4]: [channel#112, i_brand_id#105, i_class_id#106, i_category_id#107] +Functions [2]: [sum(sales#113), sum(number_sales#114)] +Aggregate Attributes [2]: [sum(sales#113)#101, sum(number_sales#114)#102] +Results [5]: [channel#112, i_brand_id#105, i_class_id#106, sum(sales#113)#101 AS sum_sales#141, sum(number_sales#114)#102 AS number_sales#142] + +(118) HashAggregate [codegen id : 230] +Input [5]: [channel#112, i_brand_id#105, i_class_id#106, sum_sales#141, number_sales#142] +Keys [3]: [channel#112, i_brand_id#105, i_class_id#106] +Functions [2]: [partial_sum(sum_sales#141), partial_sum(number_sales#142)] +Aggregate Attributes [3]: [sum#143, isEmpty#144, sum#145] +Results [6]: [channel#112, i_brand_id#105, i_class_id#106, sum#146, isEmpty#147, sum#148] + +(119) Exchange +Input [6]: [channel#112, i_brand_id#105, i_class_id#106, sum#146, isEmpty#147, sum#148] +Arguments: hashpartitioning(channel#112, i_brand_id#105, i_class_id#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(120) HashAggregate [codegen id : 231] +Input [6]: [channel#112, i_brand_id#105, i_class_id#106, sum#146, isEmpty#147, sum#148] +Keys [3]: [channel#112, i_brand_id#105, i_class_id#106] +Functions [2]: [sum(sum_sales#141), sum(number_sales#142)] +Aggregate Attributes [2]: [sum(sum_sales#141)#149, sum(number_sales#142)#150] +Results [6]: [channel#112, i_brand_id#105, i_class_id#106, null AS i_category_id#151, sum(sum_sales#141)#149 AS sum(sum_sales)#152, sum(number_sales#142)#150 AS sum(number_sales)#153] + +(121) ReusedExchange [Reuses operator id: 70] +Output [6]: [i_brand_id#154, i_class_id#155, i_category_id#156, sum#157, isEmpty#158, count#46] + +(122) HashAggregate [codegen id : 269] +Input [6]: [i_brand_id#154, i_class_id#155, i_category_id#156, sum#157, isEmpty#158, count#46] +Keys [3]: [i_brand_id#154, i_class_id#155, i_category_id#156] +Functions [2]: [sum((cast(ss_quantity#159 as decimal(10,0)) * ss_list_price#160)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#159 as decimal(10,0)) * ss_list_price#160))#47, count(1)#48] +Results [6]: [store AS channel#161, i_brand_id#154, i_class_id#155, i_category_id#156, sum((cast(ss_quantity#159 as decimal(10,0)) * ss_list_price#160))#47 AS sales#162, count(1)#48 AS number_sales#163] + +(123) Filter [codegen id : 269] +Input [6]: [channel#161, i_brand_id#154, i_class_id#155, i_category_id#156, sales#162, number_sales#163] +Condition : (isnotnull(sales#162) AND (cast(sales#162 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#12] as decimal(32,6)))) + +(124) ReusedExchange [Reuses operator id: 85] +Output [6]: [i_brand_id#164, i_class_id#165, i_category_id#166, sum#167, isEmpty#168, count#68] + +(125) HashAggregate [codegen id : 307] +Input [6]: [i_brand_id#164, i_class_id#165, i_category_id#166, sum#167, isEmpty#168, count#68] +Keys [3]: [i_brand_id#164, i_class_id#165, i_category_id#166] +Functions [2]: [sum((cast(cs_quantity#169 as decimal(10,0)) * cs_list_price#170)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#169 as decimal(10,0)) * cs_list_price#170))#69, count(1)#70] +Results [6]: [catalog AS channel#171, i_brand_id#164, i_class_id#165, i_category_id#166, sum((cast(cs_quantity#169 as decimal(10,0)) * cs_list_price#170))#69 AS sales#172, count(1)#70 AS number_sales#173] + +(126) Filter [codegen id : 307] +Input [6]: [channel#171, i_brand_id#164, i_class_id#165, i_category_id#166, sales#172, number_sales#173] +Condition : (isnotnull(sales#172) AND (cast(sales#172 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#12] as decimal(32,6)))) + +(127) ReusedExchange [Reuses operator id: 100] +Output [6]: [i_brand_id#174, i_class_id#175, i_category_id#176, sum#177, isEmpty#178, count#89] + +(128) HashAggregate [codegen id : 345] +Input [6]: [i_brand_id#174, i_class_id#175, i_category_id#176, sum#177, isEmpty#178, count#89] +Keys [3]: [i_brand_id#174, i_class_id#175, i_category_id#176] +Functions [2]: [sum((cast(ws_quantity#179 as decimal(10,0)) * ws_list_price#180)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#179 as decimal(10,0)) * ws_list_price#180))#90, count(1)#91] +Results [6]: [web AS channel#181, i_brand_id#174, i_class_id#175, i_category_id#176, sum((cast(ws_quantity#179 as decimal(10,0)) * ws_list_price#180))#90 AS sales#182, count(1)#91 AS number_sales#183] + +(129) Filter [codegen id : 345] +Input [6]: [channel#181, i_brand_id#174, i_class_id#175, i_category_id#176, sales#182, number_sales#183] +Condition : (isnotnull(sales#182) AND (cast(sales#182 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#12] as decimal(32,6)))) + +(130) Union + +(131) HashAggregate [codegen id : 346] +Input [6]: [channel#161, i_brand_id#154, i_class_id#155, i_category_id#156, sales#162, number_sales#163] +Keys [4]: [channel#161, i_brand_id#154, i_class_id#155, i_category_id#156] +Functions [2]: [partial_sum(sales#162), partial_sum(number_sales#163)] +Aggregate Attributes [3]: [sum#184, isEmpty#185, sum#186] +Results [7]: [channel#161, i_brand_id#154, i_class_id#155, i_category_id#156, sum#187, isEmpty#188, sum#189] + +(132) HashAggregate [codegen id : 346] +Input [7]: [channel#161, i_brand_id#154, i_class_id#155, i_category_id#156, sum#187, isEmpty#188, sum#189] +Keys [4]: [channel#161, i_brand_id#154, i_class_id#155, i_category_id#156] +Functions [2]: [sum(sales#162), sum(number_sales#163)] +Aggregate Attributes [2]: [sum(sales#162)#101, sum(number_sales#163)#102] +Results [4]: [channel#161, i_brand_id#154, sum(sales#162)#101 AS sum_sales#190, sum(number_sales#163)#102 AS number_sales#191] + +(133) HashAggregate [codegen id : 346] +Input [4]: [channel#161, i_brand_id#154, sum_sales#190, number_sales#191] +Keys [2]: [channel#161, i_brand_id#154] +Functions [2]: [partial_sum(sum_sales#190), partial_sum(number_sales#191)] +Aggregate Attributes [3]: [sum#192, isEmpty#193, sum#194] +Results [5]: [channel#161, i_brand_id#154, sum#195, isEmpty#196, sum#197] + +(134) Exchange +Input [5]: [channel#161, i_brand_id#154, sum#195, isEmpty#196, sum#197] +Arguments: hashpartitioning(channel#161, i_brand_id#154, 5), ENSURE_REQUIREMENTS, [plan_id=16] + +(135) HashAggregate [codegen id : 347] +Input [5]: [channel#161, i_brand_id#154, sum#195, isEmpty#196, sum#197] +Keys [2]: [channel#161, i_brand_id#154] +Functions [2]: [sum(sum_sales#190), sum(number_sales#191)] +Aggregate Attributes [2]: [sum(sum_sales#190)#198, sum(number_sales#191)#199] +Results [6]: [channel#161, i_brand_id#154, null AS i_class_id#200, null AS i_category_id#201, sum(sum_sales#190)#198 AS sum(sum_sales)#202, sum(number_sales#191)#199 AS sum(number_sales)#203] + +(136) ReusedExchange [Reuses operator id: 70] +Output [6]: [i_brand_id#204, i_class_id#205, i_category_id#206, sum#207, isEmpty#208, count#46] + +(137) HashAggregate [codegen id : 385] +Input [6]: [i_brand_id#204, i_class_id#205, i_category_id#206, sum#207, isEmpty#208, count#46] +Keys [3]: [i_brand_id#204, i_class_id#205, i_category_id#206] +Functions [2]: [sum((cast(ss_quantity#209 as decimal(10,0)) * ss_list_price#210)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#209 as decimal(10,0)) * ss_list_price#210))#47, count(1)#48] +Results [6]: [store AS channel#211, i_brand_id#204, i_class_id#205, i_category_id#206, sum((cast(ss_quantity#209 as decimal(10,0)) * ss_list_price#210))#47 AS sales#212, count(1)#48 AS number_sales#213] + +(138) Filter [codegen id : 385] +Input [6]: [channel#211, i_brand_id#204, i_class_id#205, i_category_id#206, sales#212, number_sales#213] +Condition : (isnotnull(sales#212) AND (cast(sales#212 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#12] as decimal(32,6)))) + +(139) ReusedExchange [Reuses operator id: 85] +Output [6]: [i_brand_id#214, i_class_id#215, i_category_id#216, sum#217, isEmpty#218, count#68] + +(140) HashAggregate [codegen id : 423] +Input [6]: [i_brand_id#214, i_class_id#215, i_category_id#216, sum#217, isEmpty#218, count#68] +Keys [3]: [i_brand_id#214, i_class_id#215, i_category_id#216] +Functions [2]: [sum((cast(cs_quantity#219 as decimal(10,0)) * cs_list_price#220)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#219 as decimal(10,0)) * cs_list_price#220))#69, count(1)#70] +Results [6]: [catalog AS channel#221, i_brand_id#214, i_class_id#215, i_category_id#216, sum((cast(cs_quantity#219 as decimal(10,0)) * cs_list_price#220))#69 AS sales#222, count(1)#70 AS number_sales#223] + +(141) Filter [codegen id : 423] +Input [6]: [channel#221, i_brand_id#214, i_class_id#215, i_category_id#216, sales#222, number_sales#223] +Condition : (isnotnull(sales#222) AND (cast(sales#222 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#12] as decimal(32,6)))) + +(142) ReusedExchange [Reuses operator id: 100] +Output [6]: [i_brand_id#224, i_class_id#225, i_category_id#226, sum#227, isEmpty#228, count#89] + +(143) HashAggregate [codegen id : 461] +Input [6]: [i_brand_id#224, i_class_id#225, i_category_id#226, sum#227, isEmpty#228, count#89] +Keys [3]: [i_brand_id#224, i_class_id#225, i_category_id#226] +Functions [2]: [sum((cast(ws_quantity#229 as decimal(10,0)) * ws_list_price#230)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#229 as decimal(10,0)) * ws_list_price#230))#90, count(1)#91] +Results [6]: [web AS channel#231, i_brand_id#224, i_class_id#225, i_category_id#226, sum((cast(ws_quantity#229 as decimal(10,0)) * ws_list_price#230))#90 AS sales#232, count(1)#91 AS number_sales#233] + +(144) Filter [codegen id : 461] +Input [6]: [channel#231, i_brand_id#224, i_class_id#225, i_category_id#226, sales#232, number_sales#233] +Condition : (isnotnull(sales#232) AND (cast(sales#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#12] as decimal(32,6)))) + +(145) Union + +(146) HashAggregate [codegen id : 462] +Input [6]: [channel#211, i_brand_id#204, i_class_id#205, i_category_id#206, sales#212, number_sales#213] +Keys [4]: [channel#211, i_brand_id#204, i_class_id#205, i_category_id#206] +Functions [2]: [partial_sum(sales#212), partial_sum(number_sales#213)] +Aggregate Attributes [3]: [sum#234, isEmpty#235, sum#236] +Results [7]: [channel#211, i_brand_id#204, i_class_id#205, i_category_id#206, sum#237, isEmpty#238, sum#239] + +(147) HashAggregate [codegen id : 462] +Input [7]: [channel#211, i_brand_id#204, i_class_id#205, i_category_id#206, sum#237, isEmpty#238, sum#239] +Keys [4]: [channel#211, i_brand_id#204, i_class_id#205, i_category_id#206] +Functions [2]: [sum(sales#212), sum(number_sales#213)] +Aggregate Attributes [2]: [sum(sales#212)#101, sum(number_sales#213)#102] +Results [3]: [channel#211, sum(sales#212)#101 AS sum_sales#240, sum(number_sales#213)#102 AS number_sales#241] + +(148) HashAggregate [codegen id : 462] +Input [3]: [channel#211, sum_sales#240, number_sales#241] +Keys [1]: [channel#211] +Functions [2]: [partial_sum(sum_sales#240), partial_sum(number_sales#241)] +Aggregate Attributes [3]: [sum#242, isEmpty#243, sum#244] +Results [4]: [channel#211, sum#245, isEmpty#246, sum#247] + +(149) Exchange +Input [4]: [channel#211, sum#245, isEmpty#246, sum#247] +Arguments: hashpartitioning(channel#211, 5), ENSURE_REQUIREMENTS, [plan_id=17] + +(150) HashAggregate [codegen id : 463] +Input [4]: [channel#211, sum#245, isEmpty#246, sum#247] +Keys [1]: [channel#211] +Functions [2]: [sum(sum_sales#240), sum(number_sales#241)] +Aggregate Attributes [2]: [sum(sum_sales#240)#248, sum(number_sales#241)#249] +Results [6]: [channel#211, null AS i_brand_id#250, null AS i_class_id#251, null AS i_category_id#252, sum(sum_sales#240)#248 AS sum(sum_sales)#253, sum(number_sales#241)#249 AS sum(number_sales)#254] + +(151) ReusedExchange [Reuses operator id: 70] +Output [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#46] + +(152) HashAggregate [codegen id : 501] +Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#46] +Keys [3]: [i_brand_id#255, i_class_id#256, i_category_id#257] +Functions [2]: [sum((cast(ss_quantity#260 as decimal(10,0)) * ss_list_price#261)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#260 as decimal(10,0)) * ss_list_price#261))#47, count(1)#48] +Results [6]: [store AS channel#262, i_brand_id#255, i_class_id#256, i_category_id#257, sum((cast(ss_quantity#260 as decimal(10,0)) * ss_list_price#261))#47 AS sales#263, count(1)#48 AS number_sales#264] + +(153) Filter [codegen id : 501] +Input [6]: [channel#262, i_brand_id#255, i_class_id#256, i_category_id#257, sales#263, number_sales#264] +Condition : (isnotnull(sales#263) AND (cast(sales#263 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#12] as decimal(32,6)))) + +(154) ReusedExchange [Reuses operator id: 85] +Output [6]: [i_brand_id#265, i_class_id#266, i_category_id#267, sum#268, isEmpty#269, count#68] + +(155) HashAggregate [codegen id : 539] +Input [6]: [i_brand_id#265, i_class_id#266, i_category_id#267, sum#268, isEmpty#269, count#68] +Keys [3]: [i_brand_id#265, i_class_id#266, i_category_id#267] +Functions [2]: [sum((cast(cs_quantity#270 as decimal(10,0)) * cs_list_price#271)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#270 as decimal(10,0)) * cs_list_price#271))#69, count(1)#70] +Results [6]: [catalog AS channel#272, i_brand_id#265, i_class_id#266, i_category_id#267, sum((cast(cs_quantity#270 as decimal(10,0)) * cs_list_price#271))#69 AS sales#273, count(1)#70 AS number_sales#274] + +(156) Filter [codegen id : 539] +Input [6]: [channel#272, i_brand_id#265, i_class_id#266, i_category_id#267, sales#273, number_sales#274] +Condition : (isnotnull(sales#273) AND (cast(sales#273 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#12] as decimal(32,6)))) + +(157) ReusedExchange [Reuses operator id: 100] +Output [6]: [i_brand_id#275, i_class_id#276, i_category_id#277, sum#278, isEmpty#279, count#89] + +(158) HashAggregate [codegen id : 577] +Input [6]: [i_brand_id#275, i_class_id#276, i_category_id#277, sum#278, isEmpty#279, count#89] +Keys [3]: [i_brand_id#275, i_class_id#276, i_category_id#277] +Functions [2]: [sum((cast(ws_quantity#280 as decimal(10,0)) * ws_list_price#281)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#280 as decimal(10,0)) * ws_list_price#281))#90, count(1)#91] +Results [6]: [web AS channel#282, i_brand_id#275, i_class_id#276, i_category_id#277, sum((cast(ws_quantity#280 as decimal(10,0)) * ws_list_price#281))#90 AS sales#283, count(1)#91 AS number_sales#284] + +(159) Filter [codegen id : 577] +Input [6]: [channel#282, i_brand_id#275, i_class_id#276, i_category_id#277, sales#283, number_sales#284] +Condition : (isnotnull(sales#283) AND (cast(sales#283 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#12] as decimal(32,6)))) + +(160) Union + +(161) HashAggregate [codegen id : 578] +Input [6]: [channel#262, i_brand_id#255, i_class_id#256, i_category_id#257, sales#263, number_sales#264] +Keys [4]: [channel#262, i_brand_id#255, i_class_id#256, i_category_id#257] +Functions [2]: [partial_sum(sales#263), partial_sum(number_sales#264)] +Aggregate Attributes [3]: [sum#285, isEmpty#286, sum#287] +Results [7]: [channel#262, i_brand_id#255, i_class_id#256, i_category_id#257, sum#288, isEmpty#289, sum#290] + +(162) HashAggregate [codegen id : 578] +Input [7]: [channel#262, i_brand_id#255, i_class_id#256, i_category_id#257, sum#288, isEmpty#289, sum#290] +Keys [4]: [channel#262, i_brand_id#255, i_class_id#256, i_category_id#257] +Functions [2]: [sum(sales#263), sum(number_sales#264)] +Aggregate Attributes [2]: [sum(sales#263)#101, sum(number_sales#264)#102] +Results [2]: [sum(sales#263)#101 AS sum_sales#291, sum(number_sales#264)#102 AS number_sales#292] + +(163) HashAggregate [codegen id : 578] +Input [2]: [sum_sales#291, number_sales#292] Keys: [] -Functions [2]: [partial_sum(sum_sales#183), partial_sum(number_sales#184)] -Aggregate Attributes [3]: [sum#185, isEmpty#186, sum#187] -Results [3]: [sum#188, isEmpty#189, sum#190] +Functions [2]: [partial_sum(sum_sales#291), partial_sum(number_sales#292)] +Aggregate Attributes [3]: [sum#293, isEmpty#294, sum#295] +Results [3]: [sum#296, isEmpty#297, sum#298] -(125) Exchange -Input [3]: [sum#188, isEmpty#189, sum#190] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=19] +(164) Exchange +Input [3]: [sum#296, isEmpty#297, sum#298] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] -(126) HashAggregate [codegen id : 584] -Input [3]: [sum#188, isEmpty#189, sum#190] +(165) HashAggregate [codegen id : 579] +Input [3]: [sum#296, isEmpty#297, sum#298] Keys: [] -Functions [2]: [sum(sum_sales#183), sum(number_sales#184)] -Aggregate Attributes [2]: [sum(sum_sales#183)#191, sum(number_sales#184)#192] -Results [6]: [null AS channel#193, null AS i_brand_id#194, null AS i_class_id#195, null AS i_category_id#196, sum(sum_sales#183)#191 AS sum(sum_sales)#197, sum(number_sales#184)#192 AS sum(number_sales)#198] +Functions [2]: [sum(sum_sales#291), sum(number_sales#292)] +Aggregate Attributes [2]: [sum(sum_sales#291)#299, sum(number_sales#292)#300] +Results [6]: [null AS channel#301, null AS i_brand_id#302, null AS i_class_id#303, null AS i_category_id#304, sum(sum_sales#291)#299 AS sum(sum_sales)#305, sum(number_sales#292)#300 AS sum(number_sales)#306] -(127) Union +(166) Union -(128) HashAggregate [codegen id : 585] +(167) HashAggregate [codegen id : 580] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#103, number_sales#104] Keys [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#103, number_sales#104] Functions: [] Aggregate Attributes: [] Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#103, number_sales#104] -(129) Exchange +(168) Exchange Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#103, number_sales#104] -Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#103, number_sales#104, 5), ENSURE_REQUIREMENTS, [plan_id=20] +Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#103, number_sales#104, 5), ENSURE_REQUIREMENTS, [plan_id=19] -(130) HashAggregate [codegen id : 586] +(169) HashAggregate [codegen id : 581] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#103, number_sales#104] Keys [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#103, number_sales#104] Functions: [] Aggregate Attributes: [] Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#103, number_sales#104] -(131) TakeOrderedAndProject +(170) TakeOrderedAndProject Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#103, number_sales#104] Arguments: 100, [channel#49 ASC NULLS FIRST, i_brand_id#38 ASC NULLS FIRST, i_class_id#39 ASC NULLS FIRST, i_category_id#40 ASC NULLS FIRST], [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#103, number_sales#104] ===== Subqueries ===== Subquery:1 Hosting operator id = 72 Hosting Expression = Subquery scalar-subquery#52, [id=#12] -* HashAggregate (150) -+- Exchange (149) - +- * HashAggregate (148) - +- Union (147) - :- * Project (136) - : +- * BroadcastHashJoin Inner BuildRight (135) - : :- * ColumnarToRow (133) - : : +- Scan parquet spark_catalog.default.store_sales (132) - : +- ReusedExchange (134) - :- * Project (141) - : +- * BroadcastHashJoin Inner BuildRight (140) - : :- * ColumnarToRow (138) - : : +- Scan parquet spark_catalog.default.catalog_sales (137) - : +- ReusedExchange (139) - +- * Project (146) - +- * BroadcastHashJoin Inner BuildRight (145) - :- * ColumnarToRow (143) - : +- Scan parquet spark_catalog.default.web_sales (142) - +- ReusedExchange (144) - - -(132) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#199, ss_list_price#200, ss_sold_date_sk#201] +* HashAggregate (189) ++- Exchange (188) + +- * HashAggregate (187) + +- Union (186) + :- * Project (175) + : +- * BroadcastHashJoin Inner BuildRight (174) + : :- * ColumnarToRow (172) + : : +- Scan parquet spark_catalog.default.store_sales (171) + : +- ReusedExchange (173) + :- * Project (180) + : +- * BroadcastHashJoin Inner BuildRight (179) + : :- * ColumnarToRow (177) + : : +- Scan parquet spark_catalog.default.catalog_sales (176) + : +- ReusedExchange (178) + +- * Project (185) + +- * BroadcastHashJoin Inner BuildRight (184) + :- * ColumnarToRow (182) + : +- Scan parquet spark_catalog.default.web_sales (181) + +- ReusedExchange (183) + + +(171) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#307, ss_list_price#308, ss_sold_date_sk#309] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#201), dynamicpruningexpression(ss_sold_date_sk#201 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#309), dynamicpruningexpression(ss_sold_date_sk#309 IN dynamicpruning#12)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#199, ss_list_price#200, ss_sold_date_sk#201] +(172) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#307, ss_list_price#308, ss_sold_date_sk#309] -(134) ReusedExchange [Reuses operator id: 165] -Output [1]: [d_date_sk#202] +(173) ReusedExchange [Reuses operator id: 204] +Output [1]: [d_date_sk#310] -(135) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#201] -Right keys [1]: [d_date_sk#202] +(174) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#309] +Right keys [1]: [d_date_sk#310] Join type: Inner Join condition: None -(136) Project [codegen id : 2] -Output [2]: [ss_quantity#199 AS quantity#203, ss_list_price#200 AS list_price#204] -Input [4]: [ss_quantity#199, ss_list_price#200, ss_sold_date_sk#201, d_date_sk#202] +(175) Project [codegen id : 2] +Output [2]: [ss_quantity#307 AS quantity#311, ss_list_price#308 AS list_price#312] +Input [4]: [ss_quantity#307, ss_list_price#308, ss_sold_date_sk#309, d_date_sk#310] -(137) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#205, cs_list_price#206, cs_sold_date_sk#207] +(176) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#313, cs_list_price#314, cs_sold_date_sk#315] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#207), dynamicpruningexpression(cs_sold_date_sk#207 IN dynamicpruning#208)] +PartitionFilters: [isnotnull(cs_sold_date_sk#315), dynamicpruningexpression(cs_sold_date_sk#315 IN dynamicpruning#316)] ReadSchema: struct -(138) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#205, cs_list_price#206, cs_sold_date_sk#207] +(177) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#313, cs_list_price#314, cs_sold_date_sk#315] -(139) ReusedExchange [Reuses operator id: 155] -Output [1]: [d_date_sk#209] +(178) ReusedExchange [Reuses operator id: 194] +Output [1]: [d_date_sk#317] -(140) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#207] -Right keys [1]: [d_date_sk#209] +(179) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#315] +Right keys [1]: [d_date_sk#317] Join type: Inner Join condition: None -(141) Project [codegen id : 4] -Output [2]: [cs_quantity#205 AS quantity#210, cs_list_price#206 AS list_price#211] -Input [4]: [cs_quantity#205, cs_list_price#206, cs_sold_date_sk#207, d_date_sk#209] +(180) Project [codegen id : 4] +Output [2]: [cs_quantity#313 AS quantity#318, cs_list_price#314 AS list_price#319] +Input [4]: [cs_quantity#313, cs_list_price#314, cs_sold_date_sk#315, d_date_sk#317] -(142) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#212, ws_list_price#213, ws_sold_date_sk#214] +(181) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#320, ws_list_price#321, ws_sold_date_sk#322] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#214), dynamicpruningexpression(ws_sold_date_sk#214 IN dynamicpruning#208)] +PartitionFilters: [isnotnull(ws_sold_date_sk#322), dynamicpruningexpression(ws_sold_date_sk#322 IN dynamicpruning#316)] ReadSchema: struct -(143) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#212, ws_list_price#213, ws_sold_date_sk#214] +(182) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#320, ws_list_price#321, ws_sold_date_sk#322] -(144) ReusedExchange [Reuses operator id: 155] -Output [1]: [d_date_sk#215] +(183) ReusedExchange [Reuses operator id: 194] +Output [1]: [d_date_sk#323] -(145) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#214] -Right keys [1]: [d_date_sk#215] +(184) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#322] +Right keys [1]: [d_date_sk#323] Join type: Inner Join condition: None -(146) Project [codegen id : 6] -Output [2]: [ws_quantity#212 AS quantity#216, ws_list_price#213 AS list_price#217] -Input [4]: [ws_quantity#212, ws_list_price#213, ws_sold_date_sk#214, d_date_sk#215] +(185) Project [codegen id : 6] +Output [2]: [ws_quantity#320 AS quantity#324, ws_list_price#321 AS list_price#325] +Input [4]: [ws_quantity#320, ws_list_price#321, ws_sold_date_sk#322, d_date_sk#323] -(147) Union +(186) Union -(148) HashAggregate [codegen id : 7] -Input [2]: [quantity#203, list_price#204] +(187) HashAggregate [codegen id : 7] +Input [2]: [quantity#311, list_price#312] Keys: [] -Functions [1]: [partial_avg((cast(quantity#203 as decimal(10,0)) * list_price#204))] -Aggregate Attributes [2]: [sum#218, count#219] -Results [2]: [sum#220, count#221] +Functions [1]: [partial_avg((cast(quantity#311 as decimal(10,0)) * list_price#312))] +Aggregate Attributes [2]: [sum#326, count#327] +Results [2]: [sum#328, count#329] -(149) Exchange -Input [2]: [sum#220, count#221] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=21] +(188) Exchange +Input [2]: [sum#328, count#329] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=20] -(150) HashAggregate [codegen id : 8] -Input [2]: [sum#220, count#221] +(189) HashAggregate [codegen id : 8] +Input [2]: [sum#328, count#329] Keys: [] -Functions [1]: [avg((cast(quantity#203 as decimal(10,0)) * list_price#204))] -Aggregate Attributes [1]: [avg((cast(quantity#203 as decimal(10,0)) * list_price#204))#222] -Results [1]: [avg((cast(quantity#203 as decimal(10,0)) * list_price#204))#222 AS average_sales#223] +Functions [1]: [avg((cast(quantity#311 as decimal(10,0)) * list_price#312))] +Aggregate Attributes [1]: [avg((cast(quantity#311 as decimal(10,0)) * list_price#312))#330] +Results [1]: [avg((cast(quantity#311 as decimal(10,0)) * list_price#312))#330 AS average_sales#331] -Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#201 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 171 Hosting Expression = ss_sold_date_sk#309 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 137 Hosting Expression = cs_sold_date_sk#207 IN dynamicpruning#208 -BroadcastExchange (155) -+- * Project (154) - +- * Filter (153) - +- * ColumnarToRow (152) - +- Scan parquet spark_catalog.default.date_dim (151) +Subquery:3 Hosting operator id = 176 Hosting Expression = cs_sold_date_sk#315 IN dynamicpruning#316 +BroadcastExchange (194) ++- * Project (193) + +- * Filter (192) + +- * ColumnarToRow (191) + +- Scan parquet spark_catalog.default.date_dim (190) -(151) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#209, d_year#224] +(190) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#317, d_year#332] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(152) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#209, d_year#224] +(191) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#317, d_year#332] -(153) Filter [codegen id : 1] -Input [2]: [d_date_sk#209, d_year#224] -Condition : (((isnotnull(d_year#224) AND (d_year#224 >= 1998)) AND (d_year#224 <= 2000)) AND isnotnull(d_date_sk#209)) +(192) Filter [codegen id : 1] +Input [2]: [d_date_sk#317, d_year#332] +Condition : (((isnotnull(d_year#332) AND (d_year#332 >= 1998)) AND (d_year#332 <= 2000)) AND isnotnull(d_date_sk#317)) -(154) Project [codegen id : 1] -Output [1]: [d_date_sk#209] -Input [2]: [d_date_sk#209, d_year#224] +(193) Project [codegen id : 1] +Output [1]: [d_date_sk#317] +Input [2]: [d_date_sk#317, d_year#332] -(155) BroadcastExchange -Input [1]: [d_date_sk#209] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=22] +(194) BroadcastExchange +Input [1]: [d_date_sk#317] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] -Subquery:4 Hosting operator id = 142 Hosting Expression = ws_sold_date_sk#214 IN dynamicpruning#208 +Subquery:4 Hosting operator id = 181 Hosting Expression = ws_sold_date_sk#322 IN dynamicpruning#316 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (160) -+- * Project (159) - +- * Filter (158) - +- * ColumnarToRow (157) - +- Scan parquet spark_catalog.default.date_dim (156) +BroadcastExchange (199) ++- * Project (198) + +- * Filter (197) + +- * ColumnarToRow (196) + +- Scan parquet spark_catalog.default.date_dim (195) -(156) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#36, d_year#225, d_moy#226] +(195) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#36, d_year#333, d_moy#334] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct -(157) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#36, d_year#225, d_moy#226] +(196) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#36, d_year#333, d_moy#334] -(158) Filter [codegen id : 1] -Input [3]: [d_date_sk#36, d_year#225, d_moy#226] -Condition : ((((isnotnull(d_year#225) AND isnotnull(d_moy#226)) AND (d_year#225 = 2000)) AND (d_moy#226 = 11)) AND isnotnull(d_date_sk#36)) +(197) Filter [codegen id : 1] +Input [3]: [d_date_sk#36, d_year#333, d_moy#334] +Condition : ((((isnotnull(d_year#333) AND isnotnull(d_moy#334)) AND (d_year#333 = 2000)) AND (d_moy#334 = 11)) AND isnotnull(d_date_sk#36)) -(159) Project [codegen id : 1] +(198) Project [codegen id : 1] Output [1]: [d_date_sk#36] -Input [3]: [d_date_sk#36, d_year#225, d_moy#226] +Input [3]: [d_date_sk#36, d_year#333, d_moy#334] -(160) BroadcastExchange +(199) BroadcastExchange Input [1]: [d_date_sk#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=22] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (165) -+- * Project (164) - +- * Filter (163) - +- * ColumnarToRow (162) - +- Scan parquet spark_catalog.default.date_dim (161) +BroadcastExchange (204) ++- * Project (203) + +- * Filter (202) + +- * ColumnarToRow (201) + +- Scan parquet spark_catalog.default.date_dim (200) -(161) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#227] +(200) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#13, d_year#335] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(162) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#227] +(201) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#13, d_year#335] -(163) Filter [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#227] -Condition : (((isnotnull(d_year#227) AND (d_year#227 >= 1999)) AND (d_year#227 <= 2001)) AND isnotnull(d_date_sk#13)) +(202) Filter [codegen id : 1] +Input [2]: [d_date_sk#13, d_year#335] +Condition : (((isnotnull(d_year#335) AND (d_year#335 >= 1999)) AND (d_year#335 <= 2001)) AND isnotnull(d_date_sk#13)) -(164) Project [codegen id : 1] +(203) Project [codegen id : 1] Output [1]: [d_date_sk#13] -Input [2]: [d_date_sk#13, d_year#227] +Input [2]: [d_date_sk#13, d_year#335] -(165) BroadcastExchange +(204) BroadcastExchange Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=23] Subquery:7 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12 @@ -991,4 +1218,28 @@ Subquery:11 Hosting operator id = 102 Hosting Expression = ReusedSubquery Subque Subquery:12 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 +Subquery:13 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#12] + +Subquery:14 Hosting operator id = 111 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#12] + +Subquery:15 Hosting operator id = 114 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#12] + +Subquery:16 Hosting operator id = 123 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#12] + +Subquery:17 Hosting operator id = 126 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#12] + +Subquery:18 Hosting operator id = 129 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#12] + +Subquery:19 Hosting operator id = 138 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#12] + +Subquery:20 Hosting operator id = 141 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#12] + +Subquery:21 Hosting operator id = 144 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#12] + +Subquery:22 Hosting operator id = 153 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#12] + +Subquery:23 Hosting operator id = 156 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#12] + +Subquery:24 Hosting operator id = 159 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#12] + diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index 84c70b046c7d8..795ef4fad4f1d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -1,279 +1,352 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (586) + WholeStageCodegen (581) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (585) + WholeStageCodegen (580) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (116) + WholeStageCodegen (115) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (115) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (38) - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #16 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #9 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #17 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #17 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #17 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (37) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (38) + Filter [sales] + Subquery #3 + WholeStageCodegen (8) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + InputAdapter + Exchange #15 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #8 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #16 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #16 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (17) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #16 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #2 + WholeStageCodegen (37) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (16) - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (17) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + Filter [i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (16) + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (11) + Sort [brand_id,class_id,category_id] InputAdapter - WholeStageCodegen (11) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (10) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (10) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (8) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (3) + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - WholeStageCodegen (3) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (2) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Exchange [i_brand_id,i_class_id,i_category_id] #10 + WholeStageCodegen (2) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (7) + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - WholeStageCodegen (7) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (6) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (6) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (5) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (15) + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - WholeStageCodegen (15) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (14) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (36) - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #5 - WholeStageCodegen (76) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #18 - WholeStageCodegen (75) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (14) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (36) + BroadcastHashJoin [i_item_sk,ss_item_sk] + Filter [i_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [ss_item_sk] #5 + Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #4 + WholeStageCodegen (76) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #17 + WholeStageCodegen (75) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (114) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #19 - WholeStageCodegen (113) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (114) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #18 + WholeStageCodegen (113) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (233) + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (231) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_class_id] #20 - WholeStageCodegen (232) + Exchange [channel,i_brand_id,i_class_id] #19 + WholeStageCodegen (230) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (350) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (153) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #2 + WholeStageCodegen (191) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #17 + WholeStageCodegen (229) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #18 + WholeStageCodegen (347) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id] #21 - WholeStageCodegen (349) + Exchange [channel,i_brand_id] #20 + WholeStageCodegen (346) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (467) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (269) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #2 + WholeStageCodegen (307) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #17 + WholeStageCodegen (345) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #18 + WholeStageCodegen (463) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel] #22 - WholeStageCodegen (466) + Exchange [channel] #21 + WholeStageCodegen (462) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (584) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (385) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #2 + WholeStageCodegen (423) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #17 + WholeStageCodegen (461) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #18 + WholeStageCodegen (579) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange #23 - WholeStageCodegen (583) + Exchange #22 + WholeStageCodegen (578) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (501) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #2 + WholeStageCodegen (539) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #17 + WholeStageCodegen (577) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #18 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index 4707fa56c4143..1cff6ef4600a1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -1,129 +1,168 @@ == Physical Plan == -TakeOrderedAndProject (125) -+- * HashAggregate (124) - +- Exchange (123) - +- * HashAggregate (122) - +- Union (121) - :- * HashAggregate (100) - : +- Exchange (99) - : +- * HashAggregate (98) - : +- Union (97) - : :- * Filter (66) - : : +- * HashAggregate (65) - : : +- Exchange (64) - : : +- * HashAggregate (63) - : : +- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (50) - : : : : +- * Project (49) - : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (47) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : : :- * HashAggregate (35) - : : : : : +- Exchange (34) - : : : : : +- * HashAggregate (33) - : : : : : +- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : : +- BroadcastExchange (27) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * Filter (12) - : : : : : : : +- * ColumnarToRow (11) - : : : : : : : +- Scan parquet spark_catalog.default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Filter (15) - : : : : : : : : +- * ColumnarToRow (14) - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * Filter (18) - : : : : : : : +- * ColumnarToRow (17) - : : : : : : : +- Scan parquet spark_catalog.default.item (16) - : : : : : : +- ReusedExchange (22) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (45) - : : : : +- * Project (44) - : : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : : :- * Project (41) - : : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : : :- * Filter (38) - : : : : : : +- * ColumnarToRow (37) - : : : : : : +- Scan parquet spark_catalog.default.web_sales (36) - : : : : : +- ReusedExchange (39) - : : : : +- ReusedExchange (42) - : : : +- BroadcastExchange (57) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (56) - : : : :- * Filter (54) - : : : : +- * ColumnarToRow (53) - : : : : +- Scan parquet spark_catalog.default.item (52) - : : : +- ReusedExchange (55) - : : +- ReusedExchange (60) - : :- * Filter (81) - : : +- * HashAggregate (80) - : : +- Exchange (79) - : : +- * HashAggregate (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : : : :- * Filter (69) - : : : : : +- * ColumnarToRow (68) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (67) - : : : : +- ReusedExchange (70) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- * Filter (96) - : +- * HashAggregate (95) - : +- Exchange (94) - : +- * HashAggregate (93) - : +- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * Project (89) - : : +- * BroadcastHashJoin Inner BuildRight (88) - : : :- * BroadcastHashJoin LeftSemi BuildRight (86) - : : : :- * Filter (84) - : : : : +- * ColumnarToRow (83) - : : : : +- Scan parquet spark_catalog.default.web_sales (82) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (87) - : +- ReusedExchange (90) - :- * HashAggregate (105) - : +- Exchange (104) - : +- * HashAggregate (103) - : +- * HashAggregate (102) - : +- ReusedExchange (101) - :- * HashAggregate (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- ReusedExchange (106) - :- * HashAggregate (115) - : +- Exchange (114) - : +- * HashAggregate (113) - : +- * HashAggregate (112) - : +- ReusedExchange (111) - +- * HashAggregate (120) - +- Exchange (119) - +- * HashAggregate (118) - +- * HashAggregate (117) - +- ReusedExchange (116) +TakeOrderedAndProject (164) ++- * HashAggregate (163) + +- Exchange (162) + +- * HashAggregate (161) + +- Union (160) + :- * HashAggregate (99) + : +- * HashAggregate (98) + : +- Union (97) + : :- * Filter (66) + : : +- * HashAggregate (65) + : : +- Exchange (64) + : : +- * HashAggregate (63) + : : +- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (51) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (50) + : : : : +- * Project (49) + : : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet spark_catalog.default.item (4) + : : : : +- BroadcastExchange (47) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : : :- * HashAggregate (35) + : : : : : +- Exchange (34) + : : : : : +- * HashAggregate (33) + : : : : : +- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Project (29) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : : : +- BroadcastExchange (27) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : : :- * Filter (12) + : : : : : : : +- * ColumnarToRow (11) + : : : : : : : +- Scan parquet spark_catalog.default.item (10) + : : : : : : +- BroadcastExchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Filter (15) + : : : : : : : : +- * ColumnarToRow (14) + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * Filter (18) + : : : : : : : +- * ColumnarToRow (17) + : : : : : : : +- Scan parquet spark_catalog.default.item (16) + : : : : : : +- ReusedExchange (22) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (45) + : : : : +- * Project (44) + : : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : : :- * Project (41) + : : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : : :- * Filter (38) + : : : : : : +- * ColumnarToRow (37) + : : : : : : +- Scan parquet spark_catalog.default.web_sales (36) + : : : : : +- ReusedExchange (39) + : : : : +- ReusedExchange (42) + : : : +- BroadcastExchange (57) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : : :- * Filter (54) + : : : : +- * ColumnarToRow (53) + : : : : +- Scan parquet spark_catalog.default.item (52) + : : : +- ReusedExchange (55) + : : +- ReusedExchange (60) + : :- * Filter (81) + : : +- * HashAggregate (80) + : : +- Exchange (79) + : : +- * HashAggregate (78) + : : +- * Project (77) + : : +- * BroadcastHashJoin Inner BuildRight (76) + : : :- * Project (74) + : : : +- * BroadcastHashJoin Inner BuildRight (73) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (71) + : : : : :- * Filter (69) + : : : : : +- * ColumnarToRow (68) + : : : : : +- Scan parquet spark_catalog.default.catalog_sales (67) + : : : : +- ReusedExchange (70) + : : : +- ReusedExchange (72) + : : +- ReusedExchange (75) + : +- * Filter (96) + : +- * HashAggregate (95) + : +- Exchange (94) + : +- * HashAggregate (93) + : +- * Project (92) + : +- * BroadcastHashJoin Inner BuildRight (91) + : :- * Project (89) + : : +- * BroadcastHashJoin Inner BuildRight (88) + : : :- * BroadcastHashJoin LeftSemi BuildRight (86) + : : : :- * Filter (84) + : : : : +- * ColumnarToRow (83) + : : : : +- Scan parquet spark_catalog.default.web_sales (82) + : : : +- ReusedExchange (85) + : : +- ReusedExchange (87) + : +- ReusedExchange (90) + :- * HashAggregate (114) + : +- Exchange (113) + : +- * HashAggregate (112) + : +- * HashAggregate (111) + : +- * HashAggregate (110) + : +- Union (109) + : :- * Filter (102) + : : +- * HashAggregate (101) + : : +- ReusedExchange (100) + : :- * Filter (105) + : : +- * HashAggregate (104) + : : +- ReusedExchange (103) + : +- * Filter (108) + : +- * HashAggregate (107) + : +- ReusedExchange (106) + :- * HashAggregate (129) + : +- Exchange (128) + : +- * HashAggregate (127) + : +- * HashAggregate (126) + : +- * HashAggregate (125) + : +- Union (124) + : :- * Filter (117) + : : +- * HashAggregate (116) + : : +- ReusedExchange (115) + : :- * Filter (120) + : : +- * HashAggregate (119) + : : +- ReusedExchange (118) + : +- * Filter (123) + : +- * HashAggregate (122) + : +- ReusedExchange (121) + :- * HashAggregate (144) + : +- Exchange (143) + : +- * HashAggregate (142) + : +- * HashAggregate (141) + : +- * HashAggregate (140) + : +- Union (139) + : :- * Filter (132) + : : +- * HashAggregate (131) + : : +- ReusedExchange (130) + : :- * Filter (135) + : : +- * HashAggregate (134) + : : +- ReusedExchange (133) + : +- * Filter (138) + : +- * HashAggregate (137) + : +- ReusedExchange (136) + +- * HashAggregate (159) + +- Exchange (158) + +- * HashAggregate (157) + +- * HashAggregate (156) + +- * HashAggregate (155) + +- Union (154) + :- * Filter (147) + : +- * HashAggregate (146) + : +- ReusedExchange (145) + :- * Filter (150) + : +- * HashAggregate (149) + : +- ReusedExchange (148) + +- * Filter (153) + +- * HashAggregate (152) + +- ReusedExchange (151) (1) Scan parquet spark_catalog.default.store_sales @@ -227,7 +266,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(22) ReusedExchange [Reuses operator id: 159] +(22) ReusedExchange [Reuses operator id: 198] Output [1]: [d_date_sk#23] (23) BroadcastHashJoin [codegen id : 3] @@ -264,7 +303,7 @@ Join condition: None Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 159] +(30) ReusedExchange [Reuses operator id: 198] Output [1]: [d_date_sk#24] (31) BroadcastHashJoin [codegen id : 6] @@ -323,7 +362,7 @@ Join condition: None Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(42) ReusedExchange [Reuses operator id: 159] +(42) ReusedExchange [Reuses operator id: 198] Output [1]: [d_date_sk#34] (43) BroadcastHashJoin [codegen id : 9] @@ -407,7 +446,7 @@ Join condition: None Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(60) ReusedExchange [Reuses operator id: 154] +(60) ReusedExchange [Reuses operator id: 193] Output [1]: [d_date_sk#40] (61) BroadcastHashJoin [codegen id : 25] @@ -479,7 +518,7 @@ Join condition: None Output [6]: [cs_quantity#54, cs_list_price#55, cs_sold_date_sk#56, i_brand_id#59, i_class_id#60, i_category_id#61] Input [8]: [cs_item_sk#53, cs_quantity#54, cs_list_price#55, cs_sold_date_sk#56, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61] -(75) ReusedExchange [Reuses operator id: 154] +(75) ReusedExchange [Reuses operator id: 193] Output [1]: [d_date_sk#62] (76) BroadcastHashJoin [codegen id : 51] @@ -551,7 +590,7 @@ Join condition: None Output [6]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#80, i_class_id#81, i_category_id#82] Input [8]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] -(90) ReusedExchange [Reuses operator id: 154] +(90) ReusedExchange [Reuses operator id: 193] Output [1]: [d_date_sk#83] (91) BroadcastHashJoin [codegen id : 77] @@ -595,359 +634,547 @@ Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] Aggregate Attributes [3]: [sum#95, isEmpty#96, sum#97] Results [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#98, isEmpty#99, sum#100] -(99) Exchange -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#98, isEmpty#99, sum#100] -Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(100) HashAggregate [codegen id : 80] +(99) HashAggregate [codegen id : 79] Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#98, isEmpty#99, sum#100] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#101, sum(number_sales#51)#102] Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum(sales#50)#101 AS sum_sales#103, sum(number_sales#51)#102 AS number_sales#104] -(101) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum#109, isEmpty#110, sum#111] - -(102) HashAggregate [codegen id : 160] -Input [7]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum#109, isEmpty#110, sum#111] -Keys [4]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108] -Functions [2]: [sum(sales#112), sum(number_sales#113)] -Aggregate Attributes [2]: [sum(sales#112)#101, sum(number_sales#113)#102] -Results [5]: [channel#105, i_brand_id#106, i_class_id#107, sum(sales#112)#101 AS sum_sales#114, sum(number_sales#113)#102 AS number_sales#115] - -(103) HashAggregate [codegen id : 160] -Input [5]: [channel#105, i_brand_id#106, i_class_id#107, sum_sales#114, number_sales#115] -Keys [3]: [channel#105, i_brand_id#106, i_class_id#107] -Functions [2]: [partial_sum(sum_sales#114), partial_sum(number_sales#115)] -Aggregate Attributes [3]: [sum#116, isEmpty#117, sum#118] -Results [6]: [channel#105, i_brand_id#106, i_class_id#107, sum#119, isEmpty#120, sum#121] - -(104) Exchange -Input [6]: [channel#105, i_brand_id#106, i_class_id#107, sum#119, isEmpty#120, sum#121] -Arguments: hashpartitioning(channel#105, i_brand_id#106, i_class_id#107, 5), ENSURE_REQUIREMENTS, [plan_id=14] - -(105) HashAggregate [codegen id : 161] -Input [6]: [channel#105, i_brand_id#106, i_class_id#107, sum#119, isEmpty#120, sum#121] -Keys [3]: [channel#105, i_brand_id#106, i_class_id#107] -Functions [2]: [sum(sum_sales#114), sum(number_sales#115)] -Aggregate Attributes [2]: [sum(sum_sales#114)#122, sum(number_sales#115)#123] -Results [6]: [channel#105, i_brand_id#106, i_class_id#107, null AS i_category_id#124, sum(sum_sales#114)#122 AS sum(sum_sales)#125, sum(number_sales#115)#123 AS sum(number_sales)#126] - -(106) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#127, i_brand_id#128, i_class_id#129, i_category_id#130, sum#131, isEmpty#132, sum#133] - -(107) HashAggregate [codegen id : 241] -Input [7]: [channel#127, i_brand_id#128, i_class_id#129, i_category_id#130, sum#131, isEmpty#132, sum#133] -Keys [4]: [channel#127, i_brand_id#128, i_class_id#129, i_category_id#130] -Functions [2]: [sum(sales#134), sum(number_sales#135)] -Aggregate Attributes [2]: [sum(sales#134)#101, sum(number_sales#135)#102] -Results [4]: [channel#127, i_brand_id#128, sum(sales#134)#101 AS sum_sales#136, sum(number_sales#135)#102 AS number_sales#137] - -(108) HashAggregate [codegen id : 241] -Input [4]: [channel#127, i_brand_id#128, sum_sales#136, number_sales#137] -Keys [2]: [channel#127, i_brand_id#128] -Functions [2]: [partial_sum(sum_sales#136), partial_sum(number_sales#137)] -Aggregate Attributes [3]: [sum#138, isEmpty#139, sum#140] -Results [5]: [channel#127, i_brand_id#128, sum#141, isEmpty#142, sum#143] - -(109) Exchange -Input [5]: [channel#127, i_brand_id#128, sum#141, isEmpty#142, sum#143] -Arguments: hashpartitioning(channel#127, i_brand_id#128, 5), ENSURE_REQUIREMENTS, [plan_id=15] - -(110) HashAggregate [codegen id : 242] -Input [5]: [channel#127, i_brand_id#128, sum#141, isEmpty#142, sum#143] -Keys [2]: [channel#127, i_brand_id#128] -Functions [2]: [sum(sum_sales#136), sum(number_sales#137)] -Aggregate Attributes [2]: [sum(sum_sales#136)#144, sum(number_sales#137)#145] -Results [6]: [channel#127, i_brand_id#128, null AS i_class_id#146, null AS i_category_id#147, sum(sum_sales#136)#144 AS sum(sum_sales)#148, sum(number_sales#137)#145 AS sum(number_sales)#149] - -(111) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#150, i_brand_id#151, i_class_id#152, i_category_id#153, sum#154, isEmpty#155, sum#156] - -(112) HashAggregate [codegen id : 322] -Input [7]: [channel#150, i_brand_id#151, i_class_id#152, i_category_id#153, sum#154, isEmpty#155, sum#156] -Keys [4]: [channel#150, i_brand_id#151, i_class_id#152, i_category_id#153] -Functions [2]: [sum(sales#157), sum(number_sales#158)] -Aggregate Attributes [2]: [sum(sales#157)#101, sum(number_sales#158)#102] -Results [3]: [channel#150, sum(sales#157)#101 AS sum_sales#159, sum(number_sales#158)#102 AS number_sales#160] - -(113) HashAggregate [codegen id : 322] -Input [3]: [channel#150, sum_sales#159, number_sales#160] -Keys [1]: [channel#150] -Functions [2]: [partial_sum(sum_sales#159), partial_sum(number_sales#160)] -Aggregate Attributes [3]: [sum#161, isEmpty#162, sum#163] -Results [4]: [channel#150, sum#164, isEmpty#165, sum#166] - -(114) Exchange -Input [4]: [channel#150, sum#164, isEmpty#165, sum#166] -Arguments: hashpartitioning(channel#150, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(115) HashAggregate [codegen id : 323] -Input [4]: [channel#150, sum#164, isEmpty#165, sum#166] -Keys [1]: [channel#150] -Functions [2]: [sum(sum_sales#159), sum(number_sales#160)] -Aggregate Attributes [2]: [sum(sum_sales#159)#167, sum(number_sales#160)#168] -Results [6]: [channel#150, null AS i_brand_id#169, null AS i_class_id#170, null AS i_category_id#171, sum(sum_sales#159)#167 AS sum(sum_sales)#172, sum(number_sales#160)#168 AS sum(number_sales)#173] - -(116) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#174, i_brand_id#175, i_class_id#176, i_category_id#177, sum#178, isEmpty#179, sum#180] - -(117) HashAggregate [codegen id : 403] -Input [7]: [channel#174, i_brand_id#175, i_class_id#176, i_category_id#177, sum#178, isEmpty#179, sum#180] -Keys [4]: [channel#174, i_brand_id#175, i_class_id#176, i_category_id#177] -Functions [2]: [sum(sales#181), sum(number_sales#182)] -Aggregate Attributes [2]: [sum(sales#181)#101, sum(number_sales#182)#102] -Results [2]: [sum(sales#181)#101 AS sum_sales#183, sum(number_sales#182)#102 AS number_sales#184] - -(118) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#183, number_sales#184] +(100) ReusedExchange [Reuses operator id: 64] +Output [6]: [i_brand_id#105, i_class_id#106, i_category_id#107, sum#108, isEmpty#109, count#46] + +(101) HashAggregate [codegen id : 105] +Input [6]: [i_brand_id#105, i_class_id#106, i_category_id#107, sum#108, isEmpty#109, count#46] +Keys [3]: [i_brand_id#105, i_class_id#106, i_category_id#107] +Functions [2]: [sum((cast(ss_quantity#110 as decimal(10,0)) * ss_list_price#111)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#110 as decimal(10,0)) * ss_list_price#111))#47, count(1)#48] +Results [6]: [store AS channel#112, i_brand_id#105, i_class_id#106, i_category_id#107, sum((cast(ss_quantity#110 as decimal(10,0)) * ss_list_price#111))#47 AS sales#113, count(1)#48 AS number_sales#114] + +(102) Filter [codegen id : 105] +Input [6]: [channel#112, i_brand_id#105, i_class_id#106, i_category_id#107, sales#113, number_sales#114] +Condition : (isnotnull(sales#113) AND (cast(sales#113 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#10] as decimal(32,6)))) + +(103) ReusedExchange [Reuses operator id: 79] +Output [6]: [i_brand_id#115, i_class_id#116, i_category_id#117, sum#118, isEmpty#119, count#68] + +(104) HashAggregate [codegen id : 131] +Input [6]: [i_brand_id#115, i_class_id#116, i_category_id#117, sum#118, isEmpty#119, count#68] +Keys [3]: [i_brand_id#115, i_class_id#116, i_category_id#117] +Functions [2]: [sum((cast(cs_quantity#120 as decimal(10,0)) * cs_list_price#121)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#120 as decimal(10,0)) * cs_list_price#121))#69, count(1)#70] +Results [6]: [catalog AS channel#122, i_brand_id#115, i_class_id#116, i_category_id#117, sum((cast(cs_quantity#120 as decimal(10,0)) * cs_list_price#121))#69 AS sales#123, count(1)#70 AS number_sales#124] + +(105) Filter [codegen id : 131] +Input [6]: [channel#122, i_brand_id#115, i_class_id#116, i_category_id#117, sales#123, number_sales#124] +Condition : (isnotnull(sales#123) AND (cast(sales#123 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#10] as decimal(32,6)))) + +(106) ReusedExchange [Reuses operator id: 94] +Output [6]: [i_brand_id#125, i_class_id#126, i_category_id#127, sum#128, isEmpty#129, count#89] + +(107) HashAggregate [codegen id : 157] +Input [6]: [i_brand_id#125, i_class_id#126, i_category_id#127, sum#128, isEmpty#129, count#89] +Keys [3]: [i_brand_id#125, i_class_id#126, i_category_id#127] +Functions [2]: [sum((cast(ws_quantity#130 as decimal(10,0)) * ws_list_price#131)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#130 as decimal(10,0)) * ws_list_price#131))#90, count(1)#91] +Results [6]: [web AS channel#132, i_brand_id#125, i_class_id#126, i_category_id#127, sum((cast(ws_quantity#130 as decimal(10,0)) * ws_list_price#131))#90 AS sales#133, count(1)#91 AS number_sales#134] + +(108) Filter [codegen id : 157] +Input [6]: [channel#132, i_brand_id#125, i_class_id#126, i_category_id#127, sales#133, number_sales#134] +Condition : (isnotnull(sales#133) AND (cast(sales#133 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#10] as decimal(32,6)))) + +(109) Union + +(110) HashAggregate [codegen id : 158] +Input [6]: [channel#112, i_brand_id#105, i_class_id#106, i_category_id#107, sales#113, number_sales#114] +Keys [4]: [channel#112, i_brand_id#105, i_class_id#106, i_category_id#107] +Functions [2]: [partial_sum(sales#113), partial_sum(number_sales#114)] +Aggregate Attributes [3]: [sum#135, isEmpty#136, sum#137] +Results [7]: [channel#112, i_brand_id#105, i_class_id#106, i_category_id#107, sum#138, isEmpty#139, sum#140] + +(111) HashAggregate [codegen id : 158] +Input [7]: [channel#112, i_brand_id#105, i_class_id#106, i_category_id#107, sum#138, isEmpty#139, sum#140] +Keys [4]: [channel#112, i_brand_id#105, i_class_id#106, i_category_id#107] +Functions [2]: [sum(sales#113), sum(number_sales#114)] +Aggregate Attributes [2]: [sum(sales#113)#101, sum(number_sales#114)#102] +Results [5]: [channel#112, i_brand_id#105, i_class_id#106, sum(sales#113)#101 AS sum_sales#141, sum(number_sales#114)#102 AS number_sales#142] + +(112) HashAggregate [codegen id : 158] +Input [5]: [channel#112, i_brand_id#105, i_class_id#106, sum_sales#141, number_sales#142] +Keys [3]: [channel#112, i_brand_id#105, i_class_id#106] +Functions [2]: [partial_sum(sum_sales#141), partial_sum(number_sales#142)] +Aggregate Attributes [3]: [sum#143, isEmpty#144, sum#145] +Results [6]: [channel#112, i_brand_id#105, i_class_id#106, sum#146, isEmpty#147, sum#148] + +(113) Exchange +Input [6]: [channel#112, i_brand_id#105, i_class_id#106, sum#146, isEmpty#147, sum#148] +Arguments: hashpartitioning(channel#112, i_brand_id#105, i_class_id#106, 5), ENSURE_REQUIREMENTS, [plan_id=13] + +(114) HashAggregate [codegen id : 159] +Input [6]: [channel#112, i_brand_id#105, i_class_id#106, sum#146, isEmpty#147, sum#148] +Keys [3]: [channel#112, i_brand_id#105, i_class_id#106] +Functions [2]: [sum(sum_sales#141), sum(number_sales#142)] +Aggregate Attributes [2]: [sum(sum_sales#141)#149, sum(number_sales#142)#150] +Results [6]: [channel#112, i_brand_id#105, i_class_id#106, null AS i_category_id#151, sum(sum_sales#141)#149 AS sum(sum_sales)#152, sum(number_sales#142)#150 AS sum(number_sales)#153] + +(115) ReusedExchange [Reuses operator id: 64] +Output [6]: [i_brand_id#154, i_class_id#155, i_category_id#156, sum#157, isEmpty#158, count#46] + +(116) HashAggregate [codegen id : 185] +Input [6]: [i_brand_id#154, i_class_id#155, i_category_id#156, sum#157, isEmpty#158, count#46] +Keys [3]: [i_brand_id#154, i_class_id#155, i_category_id#156] +Functions [2]: [sum((cast(ss_quantity#159 as decimal(10,0)) * ss_list_price#160)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#159 as decimal(10,0)) * ss_list_price#160))#47, count(1)#48] +Results [6]: [store AS channel#161, i_brand_id#154, i_class_id#155, i_category_id#156, sum((cast(ss_quantity#159 as decimal(10,0)) * ss_list_price#160))#47 AS sales#162, count(1)#48 AS number_sales#163] + +(117) Filter [codegen id : 185] +Input [6]: [channel#161, i_brand_id#154, i_class_id#155, i_category_id#156, sales#162, number_sales#163] +Condition : (isnotnull(sales#162) AND (cast(sales#162 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#10] as decimal(32,6)))) + +(118) ReusedExchange [Reuses operator id: 79] +Output [6]: [i_brand_id#164, i_class_id#165, i_category_id#166, sum#167, isEmpty#168, count#68] + +(119) HashAggregate [codegen id : 211] +Input [6]: [i_brand_id#164, i_class_id#165, i_category_id#166, sum#167, isEmpty#168, count#68] +Keys [3]: [i_brand_id#164, i_class_id#165, i_category_id#166] +Functions [2]: [sum((cast(cs_quantity#169 as decimal(10,0)) * cs_list_price#170)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#169 as decimal(10,0)) * cs_list_price#170))#69, count(1)#70] +Results [6]: [catalog AS channel#171, i_brand_id#164, i_class_id#165, i_category_id#166, sum((cast(cs_quantity#169 as decimal(10,0)) * cs_list_price#170))#69 AS sales#172, count(1)#70 AS number_sales#173] + +(120) Filter [codegen id : 211] +Input [6]: [channel#171, i_brand_id#164, i_class_id#165, i_category_id#166, sales#172, number_sales#173] +Condition : (isnotnull(sales#172) AND (cast(sales#172 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#10] as decimal(32,6)))) + +(121) ReusedExchange [Reuses operator id: 94] +Output [6]: [i_brand_id#174, i_class_id#175, i_category_id#176, sum#177, isEmpty#178, count#89] + +(122) HashAggregate [codegen id : 237] +Input [6]: [i_brand_id#174, i_class_id#175, i_category_id#176, sum#177, isEmpty#178, count#89] +Keys [3]: [i_brand_id#174, i_class_id#175, i_category_id#176] +Functions [2]: [sum((cast(ws_quantity#179 as decimal(10,0)) * ws_list_price#180)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#179 as decimal(10,0)) * ws_list_price#180))#90, count(1)#91] +Results [6]: [web AS channel#181, i_brand_id#174, i_class_id#175, i_category_id#176, sum((cast(ws_quantity#179 as decimal(10,0)) * ws_list_price#180))#90 AS sales#182, count(1)#91 AS number_sales#183] + +(123) Filter [codegen id : 237] +Input [6]: [channel#181, i_brand_id#174, i_class_id#175, i_category_id#176, sales#182, number_sales#183] +Condition : (isnotnull(sales#182) AND (cast(sales#182 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#10] as decimal(32,6)))) + +(124) Union + +(125) HashAggregate [codegen id : 238] +Input [6]: [channel#161, i_brand_id#154, i_class_id#155, i_category_id#156, sales#162, number_sales#163] +Keys [4]: [channel#161, i_brand_id#154, i_class_id#155, i_category_id#156] +Functions [2]: [partial_sum(sales#162), partial_sum(number_sales#163)] +Aggregate Attributes [3]: [sum#184, isEmpty#185, sum#186] +Results [7]: [channel#161, i_brand_id#154, i_class_id#155, i_category_id#156, sum#187, isEmpty#188, sum#189] + +(126) HashAggregate [codegen id : 238] +Input [7]: [channel#161, i_brand_id#154, i_class_id#155, i_category_id#156, sum#187, isEmpty#188, sum#189] +Keys [4]: [channel#161, i_brand_id#154, i_class_id#155, i_category_id#156] +Functions [2]: [sum(sales#162), sum(number_sales#163)] +Aggregate Attributes [2]: [sum(sales#162)#101, sum(number_sales#163)#102] +Results [4]: [channel#161, i_brand_id#154, sum(sales#162)#101 AS sum_sales#190, sum(number_sales#163)#102 AS number_sales#191] + +(127) HashAggregate [codegen id : 238] +Input [4]: [channel#161, i_brand_id#154, sum_sales#190, number_sales#191] +Keys [2]: [channel#161, i_brand_id#154] +Functions [2]: [partial_sum(sum_sales#190), partial_sum(number_sales#191)] +Aggregate Attributes [3]: [sum#192, isEmpty#193, sum#194] +Results [5]: [channel#161, i_brand_id#154, sum#195, isEmpty#196, sum#197] + +(128) Exchange +Input [5]: [channel#161, i_brand_id#154, sum#195, isEmpty#196, sum#197] +Arguments: hashpartitioning(channel#161, i_brand_id#154, 5), ENSURE_REQUIREMENTS, [plan_id=14] + +(129) HashAggregate [codegen id : 239] +Input [5]: [channel#161, i_brand_id#154, sum#195, isEmpty#196, sum#197] +Keys [2]: [channel#161, i_brand_id#154] +Functions [2]: [sum(sum_sales#190), sum(number_sales#191)] +Aggregate Attributes [2]: [sum(sum_sales#190)#198, sum(number_sales#191)#199] +Results [6]: [channel#161, i_brand_id#154, null AS i_class_id#200, null AS i_category_id#201, sum(sum_sales#190)#198 AS sum(sum_sales)#202, sum(number_sales#191)#199 AS sum(number_sales)#203] + +(130) ReusedExchange [Reuses operator id: 64] +Output [6]: [i_brand_id#204, i_class_id#205, i_category_id#206, sum#207, isEmpty#208, count#46] + +(131) HashAggregate [codegen id : 265] +Input [6]: [i_brand_id#204, i_class_id#205, i_category_id#206, sum#207, isEmpty#208, count#46] +Keys [3]: [i_brand_id#204, i_class_id#205, i_category_id#206] +Functions [2]: [sum((cast(ss_quantity#209 as decimal(10,0)) * ss_list_price#210)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#209 as decimal(10,0)) * ss_list_price#210))#47, count(1)#48] +Results [6]: [store AS channel#211, i_brand_id#204, i_class_id#205, i_category_id#206, sum((cast(ss_quantity#209 as decimal(10,0)) * ss_list_price#210))#47 AS sales#212, count(1)#48 AS number_sales#213] + +(132) Filter [codegen id : 265] +Input [6]: [channel#211, i_brand_id#204, i_class_id#205, i_category_id#206, sales#212, number_sales#213] +Condition : (isnotnull(sales#212) AND (cast(sales#212 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#10] as decimal(32,6)))) + +(133) ReusedExchange [Reuses operator id: 79] +Output [6]: [i_brand_id#214, i_class_id#215, i_category_id#216, sum#217, isEmpty#218, count#68] + +(134) HashAggregate [codegen id : 291] +Input [6]: [i_brand_id#214, i_class_id#215, i_category_id#216, sum#217, isEmpty#218, count#68] +Keys [3]: [i_brand_id#214, i_class_id#215, i_category_id#216] +Functions [2]: [sum((cast(cs_quantity#219 as decimal(10,0)) * cs_list_price#220)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#219 as decimal(10,0)) * cs_list_price#220))#69, count(1)#70] +Results [6]: [catalog AS channel#221, i_brand_id#214, i_class_id#215, i_category_id#216, sum((cast(cs_quantity#219 as decimal(10,0)) * cs_list_price#220))#69 AS sales#222, count(1)#70 AS number_sales#223] + +(135) Filter [codegen id : 291] +Input [6]: [channel#221, i_brand_id#214, i_class_id#215, i_category_id#216, sales#222, number_sales#223] +Condition : (isnotnull(sales#222) AND (cast(sales#222 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#10] as decimal(32,6)))) + +(136) ReusedExchange [Reuses operator id: 94] +Output [6]: [i_brand_id#224, i_class_id#225, i_category_id#226, sum#227, isEmpty#228, count#89] + +(137) HashAggregate [codegen id : 317] +Input [6]: [i_brand_id#224, i_class_id#225, i_category_id#226, sum#227, isEmpty#228, count#89] +Keys [3]: [i_brand_id#224, i_class_id#225, i_category_id#226] +Functions [2]: [sum((cast(ws_quantity#229 as decimal(10,0)) * ws_list_price#230)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#229 as decimal(10,0)) * ws_list_price#230))#90, count(1)#91] +Results [6]: [web AS channel#231, i_brand_id#224, i_class_id#225, i_category_id#226, sum((cast(ws_quantity#229 as decimal(10,0)) * ws_list_price#230))#90 AS sales#232, count(1)#91 AS number_sales#233] + +(138) Filter [codegen id : 317] +Input [6]: [channel#231, i_brand_id#224, i_class_id#225, i_category_id#226, sales#232, number_sales#233] +Condition : (isnotnull(sales#232) AND (cast(sales#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#10] as decimal(32,6)))) + +(139) Union + +(140) HashAggregate [codegen id : 318] +Input [6]: [channel#211, i_brand_id#204, i_class_id#205, i_category_id#206, sales#212, number_sales#213] +Keys [4]: [channel#211, i_brand_id#204, i_class_id#205, i_category_id#206] +Functions [2]: [partial_sum(sales#212), partial_sum(number_sales#213)] +Aggregate Attributes [3]: [sum#234, isEmpty#235, sum#236] +Results [7]: [channel#211, i_brand_id#204, i_class_id#205, i_category_id#206, sum#237, isEmpty#238, sum#239] + +(141) HashAggregate [codegen id : 318] +Input [7]: [channel#211, i_brand_id#204, i_class_id#205, i_category_id#206, sum#237, isEmpty#238, sum#239] +Keys [4]: [channel#211, i_brand_id#204, i_class_id#205, i_category_id#206] +Functions [2]: [sum(sales#212), sum(number_sales#213)] +Aggregate Attributes [2]: [sum(sales#212)#101, sum(number_sales#213)#102] +Results [3]: [channel#211, sum(sales#212)#101 AS sum_sales#240, sum(number_sales#213)#102 AS number_sales#241] + +(142) HashAggregate [codegen id : 318] +Input [3]: [channel#211, sum_sales#240, number_sales#241] +Keys [1]: [channel#211] +Functions [2]: [partial_sum(sum_sales#240), partial_sum(number_sales#241)] +Aggregate Attributes [3]: [sum#242, isEmpty#243, sum#244] +Results [4]: [channel#211, sum#245, isEmpty#246, sum#247] + +(143) Exchange +Input [4]: [channel#211, sum#245, isEmpty#246, sum#247] +Arguments: hashpartitioning(channel#211, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(144) HashAggregate [codegen id : 319] +Input [4]: [channel#211, sum#245, isEmpty#246, sum#247] +Keys [1]: [channel#211] +Functions [2]: [sum(sum_sales#240), sum(number_sales#241)] +Aggregate Attributes [2]: [sum(sum_sales#240)#248, sum(number_sales#241)#249] +Results [6]: [channel#211, null AS i_brand_id#250, null AS i_class_id#251, null AS i_category_id#252, sum(sum_sales#240)#248 AS sum(sum_sales)#253, sum(number_sales#241)#249 AS sum(number_sales)#254] + +(145) ReusedExchange [Reuses operator id: 64] +Output [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#46] + +(146) HashAggregate [codegen id : 345] +Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#46] +Keys [3]: [i_brand_id#255, i_class_id#256, i_category_id#257] +Functions [2]: [sum((cast(ss_quantity#260 as decimal(10,0)) * ss_list_price#261)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#260 as decimal(10,0)) * ss_list_price#261))#47, count(1)#48] +Results [6]: [store AS channel#262, i_brand_id#255, i_class_id#256, i_category_id#257, sum((cast(ss_quantity#260 as decimal(10,0)) * ss_list_price#261))#47 AS sales#263, count(1)#48 AS number_sales#264] + +(147) Filter [codegen id : 345] +Input [6]: [channel#262, i_brand_id#255, i_class_id#256, i_category_id#257, sales#263, number_sales#264] +Condition : (isnotnull(sales#263) AND (cast(sales#263 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#10] as decimal(32,6)))) + +(148) ReusedExchange [Reuses operator id: 79] +Output [6]: [i_brand_id#265, i_class_id#266, i_category_id#267, sum#268, isEmpty#269, count#68] + +(149) HashAggregate [codegen id : 371] +Input [6]: [i_brand_id#265, i_class_id#266, i_category_id#267, sum#268, isEmpty#269, count#68] +Keys [3]: [i_brand_id#265, i_class_id#266, i_category_id#267] +Functions [2]: [sum((cast(cs_quantity#270 as decimal(10,0)) * cs_list_price#271)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#270 as decimal(10,0)) * cs_list_price#271))#69, count(1)#70] +Results [6]: [catalog AS channel#272, i_brand_id#265, i_class_id#266, i_category_id#267, sum((cast(cs_quantity#270 as decimal(10,0)) * cs_list_price#271))#69 AS sales#273, count(1)#70 AS number_sales#274] + +(150) Filter [codegen id : 371] +Input [6]: [channel#272, i_brand_id#265, i_class_id#266, i_category_id#267, sales#273, number_sales#274] +Condition : (isnotnull(sales#273) AND (cast(sales#273 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#10] as decimal(32,6)))) + +(151) ReusedExchange [Reuses operator id: 94] +Output [6]: [i_brand_id#275, i_class_id#276, i_category_id#277, sum#278, isEmpty#279, count#89] + +(152) HashAggregate [codegen id : 397] +Input [6]: [i_brand_id#275, i_class_id#276, i_category_id#277, sum#278, isEmpty#279, count#89] +Keys [3]: [i_brand_id#275, i_class_id#276, i_category_id#277] +Functions [2]: [sum((cast(ws_quantity#280 as decimal(10,0)) * ws_list_price#281)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#280 as decimal(10,0)) * ws_list_price#281))#90, count(1)#91] +Results [6]: [web AS channel#282, i_brand_id#275, i_class_id#276, i_category_id#277, sum((cast(ws_quantity#280 as decimal(10,0)) * ws_list_price#281))#90 AS sales#283, count(1)#91 AS number_sales#284] + +(153) Filter [codegen id : 397] +Input [6]: [channel#282, i_brand_id#275, i_class_id#276, i_category_id#277, sales#283, number_sales#284] +Condition : (isnotnull(sales#283) AND (cast(sales#283 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#10] as decimal(32,6)))) + +(154) Union + +(155) HashAggregate [codegen id : 398] +Input [6]: [channel#262, i_brand_id#255, i_class_id#256, i_category_id#257, sales#263, number_sales#264] +Keys [4]: [channel#262, i_brand_id#255, i_class_id#256, i_category_id#257] +Functions [2]: [partial_sum(sales#263), partial_sum(number_sales#264)] +Aggregate Attributes [3]: [sum#285, isEmpty#286, sum#287] +Results [7]: [channel#262, i_brand_id#255, i_class_id#256, i_category_id#257, sum#288, isEmpty#289, sum#290] + +(156) HashAggregate [codegen id : 398] +Input [7]: [channel#262, i_brand_id#255, i_class_id#256, i_category_id#257, sum#288, isEmpty#289, sum#290] +Keys [4]: [channel#262, i_brand_id#255, i_class_id#256, i_category_id#257] +Functions [2]: [sum(sales#263), sum(number_sales#264)] +Aggregate Attributes [2]: [sum(sales#263)#101, sum(number_sales#264)#102] +Results [2]: [sum(sales#263)#101 AS sum_sales#291, sum(number_sales#264)#102 AS number_sales#292] + +(157) HashAggregate [codegen id : 398] +Input [2]: [sum_sales#291, number_sales#292] Keys: [] -Functions [2]: [partial_sum(sum_sales#183), partial_sum(number_sales#184)] -Aggregate Attributes [3]: [sum#185, isEmpty#186, sum#187] -Results [3]: [sum#188, isEmpty#189, sum#190] +Functions [2]: [partial_sum(sum_sales#291), partial_sum(number_sales#292)] +Aggregate Attributes [3]: [sum#293, isEmpty#294, sum#295] +Results [3]: [sum#296, isEmpty#297, sum#298] -(119) Exchange -Input [3]: [sum#188, isEmpty#189, sum#190] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17] +(158) Exchange +Input [3]: [sum#296, isEmpty#297, sum#298] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] -(120) HashAggregate [codegen id : 404] -Input [3]: [sum#188, isEmpty#189, sum#190] +(159) HashAggregate [codegen id : 399] +Input [3]: [sum#296, isEmpty#297, sum#298] Keys: [] -Functions [2]: [sum(sum_sales#183), sum(number_sales#184)] -Aggregate Attributes [2]: [sum(sum_sales#183)#191, sum(number_sales#184)#192] -Results [6]: [null AS channel#193, null AS i_brand_id#194, null AS i_class_id#195, null AS i_category_id#196, sum(sum_sales#183)#191 AS sum(sum_sales)#197, sum(number_sales#184)#192 AS sum(number_sales)#198] +Functions [2]: [sum(sum_sales#291), sum(number_sales#292)] +Aggregate Attributes [2]: [sum(sum_sales#291)#299, sum(number_sales#292)#300] +Results [6]: [null AS channel#301, null AS i_brand_id#302, null AS i_class_id#303, null AS i_category_id#304, sum(sum_sales#291)#299 AS sum(sum_sales)#305, sum(number_sales#292)#300 AS sum(number_sales)#306] -(121) Union +(160) Union -(122) HashAggregate [codegen id : 405] +(161) HashAggregate [codegen id : 400] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#103, number_sales#104] Keys [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#103, number_sales#104] Functions: [] Aggregate Attributes: [] Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#103, number_sales#104] -(123) Exchange +(162) Exchange Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#103, number_sales#104] -Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#103, number_sales#104, 5), ENSURE_REQUIREMENTS, [plan_id=18] +Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#103, number_sales#104, 5), ENSURE_REQUIREMENTS, [plan_id=17] -(124) HashAggregate [codegen id : 406] +(163) HashAggregate [codegen id : 401] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#103, number_sales#104] Keys [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#103, number_sales#104] Functions: [] Aggregate Attributes: [] Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#103, number_sales#104] -(125) TakeOrderedAndProject +(164) TakeOrderedAndProject Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#103, number_sales#104] Arguments: 100, [channel#49 ASC NULLS FIRST, i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#103, number_sales#104] ===== Subqueries ===== Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#52, [id=#10] -* HashAggregate (144) -+- Exchange (143) - +- * HashAggregate (142) - +- Union (141) - :- * Project (130) - : +- * BroadcastHashJoin Inner BuildRight (129) - : :- * ColumnarToRow (127) - : : +- Scan parquet spark_catalog.default.store_sales (126) - : +- ReusedExchange (128) - :- * Project (135) - : +- * BroadcastHashJoin Inner BuildRight (134) - : :- * ColumnarToRow (132) - : : +- Scan parquet spark_catalog.default.catalog_sales (131) - : +- ReusedExchange (133) - +- * Project (140) - +- * BroadcastHashJoin Inner BuildRight (139) - :- * ColumnarToRow (137) - : +- Scan parquet spark_catalog.default.web_sales (136) - +- ReusedExchange (138) - - -(126) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#199, ss_list_price#200, ss_sold_date_sk#201] +* HashAggregate (183) ++- Exchange (182) + +- * HashAggregate (181) + +- Union (180) + :- * Project (169) + : +- * BroadcastHashJoin Inner BuildRight (168) + : :- * ColumnarToRow (166) + : : +- Scan parquet spark_catalog.default.store_sales (165) + : +- ReusedExchange (167) + :- * Project (174) + : +- * BroadcastHashJoin Inner BuildRight (173) + : :- * ColumnarToRow (171) + : : +- Scan parquet spark_catalog.default.catalog_sales (170) + : +- ReusedExchange (172) + +- * Project (179) + +- * BroadcastHashJoin Inner BuildRight (178) + :- * ColumnarToRow (176) + : +- Scan parquet spark_catalog.default.web_sales (175) + +- ReusedExchange (177) + + +(165) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#307, ss_list_price#308, ss_sold_date_sk#309] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#201), dynamicpruningexpression(ss_sold_date_sk#201 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#309), dynamicpruningexpression(ss_sold_date_sk#309 IN dynamicpruning#12)] ReadSchema: struct -(127) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#199, ss_list_price#200, ss_sold_date_sk#201] +(166) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#307, ss_list_price#308, ss_sold_date_sk#309] -(128) ReusedExchange [Reuses operator id: 159] -Output [1]: [d_date_sk#202] +(167) ReusedExchange [Reuses operator id: 198] +Output [1]: [d_date_sk#310] -(129) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#201] -Right keys [1]: [d_date_sk#202] +(168) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#309] +Right keys [1]: [d_date_sk#310] Join type: Inner Join condition: None -(130) Project [codegen id : 2] -Output [2]: [ss_quantity#199 AS quantity#203, ss_list_price#200 AS list_price#204] -Input [4]: [ss_quantity#199, ss_list_price#200, ss_sold_date_sk#201, d_date_sk#202] +(169) Project [codegen id : 2] +Output [2]: [ss_quantity#307 AS quantity#311, ss_list_price#308 AS list_price#312] +Input [4]: [ss_quantity#307, ss_list_price#308, ss_sold_date_sk#309, d_date_sk#310] -(131) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#205, cs_list_price#206, cs_sold_date_sk#207] +(170) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#313, cs_list_price#314, cs_sold_date_sk#315] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#207), dynamicpruningexpression(cs_sold_date_sk#207 IN dynamicpruning#208)] +PartitionFilters: [isnotnull(cs_sold_date_sk#315), dynamicpruningexpression(cs_sold_date_sk#315 IN dynamicpruning#316)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#205, cs_list_price#206, cs_sold_date_sk#207] +(171) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#313, cs_list_price#314, cs_sold_date_sk#315] -(133) ReusedExchange [Reuses operator id: 149] -Output [1]: [d_date_sk#209] +(172) ReusedExchange [Reuses operator id: 188] +Output [1]: [d_date_sk#317] -(134) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#207] -Right keys [1]: [d_date_sk#209] +(173) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#315] +Right keys [1]: [d_date_sk#317] Join type: Inner Join condition: None -(135) Project [codegen id : 4] -Output [2]: [cs_quantity#205 AS quantity#210, cs_list_price#206 AS list_price#211] -Input [4]: [cs_quantity#205, cs_list_price#206, cs_sold_date_sk#207, d_date_sk#209] +(174) Project [codegen id : 4] +Output [2]: [cs_quantity#313 AS quantity#318, cs_list_price#314 AS list_price#319] +Input [4]: [cs_quantity#313, cs_list_price#314, cs_sold_date_sk#315, d_date_sk#317] -(136) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#212, ws_list_price#213, ws_sold_date_sk#214] +(175) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#320, ws_list_price#321, ws_sold_date_sk#322] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#214), dynamicpruningexpression(ws_sold_date_sk#214 IN dynamicpruning#208)] +PartitionFilters: [isnotnull(ws_sold_date_sk#322), dynamicpruningexpression(ws_sold_date_sk#322 IN dynamicpruning#316)] ReadSchema: struct -(137) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#212, ws_list_price#213, ws_sold_date_sk#214] +(176) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#320, ws_list_price#321, ws_sold_date_sk#322] -(138) ReusedExchange [Reuses operator id: 149] -Output [1]: [d_date_sk#215] +(177) ReusedExchange [Reuses operator id: 188] +Output [1]: [d_date_sk#323] -(139) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#214] -Right keys [1]: [d_date_sk#215] +(178) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#322] +Right keys [1]: [d_date_sk#323] Join type: Inner Join condition: None -(140) Project [codegen id : 6] -Output [2]: [ws_quantity#212 AS quantity#216, ws_list_price#213 AS list_price#217] -Input [4]: [ws_quantity#212, ws_list_price#213, ws_sold_date_sk#214, d_date_sk#215] +(179) Project [codegen id : 6] +Output [2]: [ws_quantity#320 AS quantity#324, ws_list_price#321 AS list_price#325] +Input [4]: [ws_quantity#320, ws_list_price#321, ws_sold_date_sk#322, d_date_sk#323] -(141) Union +(180) Union -(142) HashAggregate [codegen id : 7] -Input [2]: [quantity#203, list_price#204] +(181) HashAggregate [codegen id : 7] +Input [2]: [quantity#311, list_price#312] Keys: [] -Functions [1]: [partial_avg((cast(quantity#203 as decimal(10,0)) * list_price#204))] -Aggregate Attributes [2]: [sum#218, count#219] -Results [2]: [sum#220, count#221] +Functions [1]: [partial_avg((cast(quantity#311 as decimal(10,0)) * list_price#312))] +Aggregate Attributes [2]: [sum#326, count#327] +Results [2]: [sum#328, count#329] -(143) Exchange -Input [2]: [sum#220, count#221] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=19] +(182) Exchange +Input [2]: [sum#328, count#329] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] -(144) HashAggregate [codegen id : 8] -Input [2]: [sum#220, count#221] +(183) HashAggregate [codegen id : 8] +Input [2]: [sum#328, count#329] Keys: [] -Functions [1]: [avg((cast(quantity#203 as decimal(10,0)) * list_price#204))] -Aggregate Attributes [1]: [avg((cast(quantity#203 as decimal(10,0)) * list_price#204))#222] -Results [1]: [avg((cast(quantity#203 as decimal(10,0)) * list_price#204))#222 AS average_sales#223] +Functions [1]: [avg((cast(quantity#311 as decimal(10,0)) * list_price#312))] +Aggregate Attributes [1]: [avg((cast(quantity#311 as decimal(10,0)) * list_price#312))#330] +Results [1]: [avg((cast(quantity#311 as decimal(10,0)) * list_price#312))#330 AS average_sales#331] -Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#201 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 165 Hosting Expression = ss_sold_date_sk#309 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 131 Hosting Expression = cs_sold_date_sk#207 IN dynamicpruning#208 -BroadcastExchange (149) -+- * Project (148) - +- * Filter (147) - +- * ColumnarToRow (146) - +- Scan parquet spark_catalog.default.date_dim (145) +Subquery:3 Hosting operator id = 170 Hosting Expression = cs_sold_date_sk#315 IN dynamicpruning#316 +BroadcastExchange (188) ++- * Project (187) + +- * Filter (186) + +- * ColumnarToRow (185) + +- Scan parquet spark_catalog.default.date_dim (184) -(145) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#209, d_year#224] +(184) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#317, d_year#332] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(146) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#209, d_year#224] +(185) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#317, d_year#332] -(147) Filter [codegen id : 1] -Input [2]: [d_date_sk#209, d_year#224] -Condition : (((isnotnull(d_year#224) AND (d_year#224 >= 1998)) AND (d_year#224 <= 2000)) AND isnotnull(d_date_sk#209)) +(186) Filter [codegen id : 1] +Input [2]: [d_date_sk#317, d_year#332] +Condition : (((isnotnull(d_year#332) AND (d_year#332 >= 1998)) AND (d_year#332 <= 2000)) AND isnotnull(d_date_sk#317)) -(148) Project [codegen id : 1] -Output [1]: [d_date_sk#209] -Input [2]: [d_date_sk#209, d_year#224] +(187) Project [codegen id : 1] +Output [1]: [d_date_sk#317] +Input [2]: [d_date_sk#317, d_year#332] -(149) BroadcastExchange -Input [1]: [d_date_sk#209] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] +(188) BroadcastExchange +Input [1]: [d_date_sk#317] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] -Subquery:4 Hosting operator id = 136 Hosting Expression = ws_sold_date_sk#214 IN dynamicpruning#208 +Subquery:4 Hosting operator id = 175 Hosting Expression = ws_sold_date_sk#322 IN dynamicpruning#316 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (154) -+- * Project (153) - +- * Filter (152) - +- * ColumnarToRow (151) - +- Scan parquet spark_catalog.default.date_dim (150) +BroadcastExchange (193) ++- * Project (192) + +- * Filter (191) + +- * ColumnarToRow (190) + +- Scan parquet spark_catalog.default.date_dim (189) -(150) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#40, d_year#225, d_moy#226] +(189) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#40, d_year#333, d_moy#334] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct -(151) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#40, d_year#225, d_moy#226] +(190) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#40, d_year#333, d_moy#334] -(152) Filter [codegen id : 1] -Input [3]: [d_date_sk#40, d_year#225, d_moy#226] -Condition : ((((isnotnull(d_year#225) AND isnotnull(d_moy#226)) AND (d_year#225 = 2000)) AND (d_moy#226 = 11)) AND isnotnull(d_date_sk#40)) +(191) Filter [codegen id : 1] +Input [3]: [d_date_sk#40, d_year#333, d_moy#334] +Condition : ((((isnotnull(d_year#333) AND isnotnull(d_moy#334)) AND (d_year#333 = 2000)) AND (d_moy#334 = 11)) AND isnotnull(d_date_sk#40)) -(153) Project [codegen id : 1] +(192) Project [codegen id : 1] Output [1]: [d_date_sk#40] -Input [3]: [d_date_sk#40, d_year#225, d_moy#226] +Input [3]: [d_date_sk#40, d_year#333, d_moy#334] -(154) BroadcastExchange +(193) BroadcastExchange Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (159) -+- * Project (158) - +- * Filter (157) - +- * ColumnarToRow (156) - +- Scan parquet spark_catalog.default.date_dim (155) +BroadcastExchange (198) ++- * Project (197) + +- * Filter (196) + +- * ColumnarToRow (195) + +- Scan parquet spark_catalog.default.date_dim (194) -(155) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#227] +(194) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#335] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(156) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#227] +(195) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#24, d_year#335] -(157) Filter [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#227] -Condition : (((isnotnull(d_year#227) AND (d_year#227 >= 1999)) AND (d_year#227 <= 2001)) AND isnotnull(d_date_sk#24)) +(196) Filter [codegen id : 1] +Input [2]: [d_date_sk#24, d_year#335] +Condition : (((isnotnull(d_year#335) AND (d_year#335 >= 1999)) AND (d_year#335 <= 2001)) AND isnotnull(d_date_sk#24)) -(158) Project [codegen id : 1] +(197) Project [codegen id : 1] Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#227] +Input [2]: [d_date_sk#24, d_year#335] -(159) BroadcastExchange +(198) BroadcastExchange Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 @@ -961,4 +1188,28 @@ Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 +Subquery:13 Hosting operator id = 102 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#10] + +Subquery:14 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#10] + +Subquery:15 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#10] + +Subquery:16 Hosting operator id = 117 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#10] + +Subquery:17 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#10] + +Subquery:18 Hosting operator id = 123 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#10] + +Subquery:19 Hosting operator id = 132 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#10] + +Subquery:20 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#10] + +Subquery:21 Hosting operator id = 138 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#10] + +Subquery:22 Hosting operator id = 147 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#10] + +Subquery:23 Hosting operator id = 150 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#10] + +Subquery:24 Hosting operator id = 153 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#10] + diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index ed3b4abc008f0..6e8a78082dbea 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -1,261 +1,334 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (406) + WholeStageCodegen (401) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (405) + WholeStageCodegen (400) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (80) + WholeStageCodegen (79) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (26) - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #14 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #15 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (26) + Filter [sales] + Subquery #3 + WholeStageCodegen (8) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + InputAdapter + Exchange #13 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #7 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #14 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #14 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #14 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #2 + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + Filter [i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (1) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Filter [cs_item_sk] + Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - ReusedExchange [ss_item_sk] #5 + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (52) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #15 + WholeStageCodegen (51) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (78) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (77) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (159) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_class_id] #18 - WholeStageCodegen (160) + Exchange [channel,i_brand_id,i_class_id] #17 + WholeStageCodegen (158) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (105) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #2 + WholeStageCodegen (131) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #15 + WholeStageCodegen (157) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #16 + WholeStageCodegen (239) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id] #19 - WholeStageCodegen (241) + Exchange [channel,i_brand_id] #18 + WholeStageCodegen (238) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (185) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #2 + WholeStageCodegen (211) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #15 + WholeStageCodegen (237) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #16 + WholeStageCodegen (319) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel] #20 - WholeStageCodegen (322) + Exchange [channel] #19 + WholeStageCodegen (318) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (265) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #2 + WholeStageCodegen (291) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #15 + WholeStageCodegen (317) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #16 + WholeStageCodegen (399) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange #21 - WholeStageCodegen (403) + Exchange #20 + WholeStageCodegen (398) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (345) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #2 + WholeStageCodegen (371) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #15 + WholeStageCodegen (397) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #16 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt index 429efb5e203c6..03aed216685fe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt @@ -1,90 +1,89 @@ == Physical Plan == -TakeOrderedAndProject (86) -+- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- Union (82) - :- * Project (27) - : +- * Filter (26) - : +- Window (25) - : +- * Sort (24) - : +- Window (23) - : +- * Sort (22) - : +- Exchange (21) - : +- * HashAggregate (20) - : +- Exchange (19) - : +- * HashAggregate (18) - : +- * Project (17) - : +- * SortMergeJoin Inner (16) - : :- * Sort (9) - : : +- Exchange (8) - : : +- * Project (7) - : : +- * BroadcastHashJoin Inner BuildRight (6) - : : :- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- ReusedExchange (5) - : +- * Sort (15) - : +- Exchange (14) - : +- * Project (13) - : +- * Filter (12) - : +- * ColumnarToRow (11) - : +- Scan parquet spark_catalog.default.web_returns (10) - :- * Project (54) - : +- * Filter (53) - : +- Window (52) - : +- * Sort (51) - : +- Window (50) - : +- * Sort (49) - : +- Exchange (48) - : +- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * SortMergeJoin Inner (43) - : :- * Sort (36) - : : +- Exchange (35) - : : +- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Project (31) - : : : +- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet spark_catalog.default.catalog_sales (28) - : : +- ReusedExchange (32) - : +- * Sort (42) - : +- Exchange (41) - : +- * Project (40) - : +- * Filter (39) - : +- * ColumnarToRow (38) - : +- Scan parquet spark_catalog.default.catalog_returns (37) - +- * Project (81) - +- * Filter (80) - +- Window (79) - +- * Sort (78) - +- Window (77) - +- * Sort (76) - +- Exchange (75) - +- * HashAggregate (74) - +- Exchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * SortMergeJoin Inner (70) - :- * Sort (63) - : +- Exchange (62) - : +- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * Project (58) - : : +- * Filter (57) - : : +- * ColumnarToRow (56) - : : +- Scan parquet spark_catalog.default.store_sales (55) - : +- ReusedExchange (59) - +- * Sort (69) - +- Exchange (68) - +- * Project (67) - +- * Filter (66) - +- * ColumnarToRow (65) - +- Scan parquet spark_catalog.default.store_returns (64) +TakeOrderedAndProject (85) ++- * HashAggregate (84) + +- * HashAggregate (83) + +- Union (82) + :- * Project (27) + : +- * Filter (26) + : +- Window (25) + : +- * Sort (24) + : +- Window (23) + : +- * Sort (22) + : +- Exchange (21) + : +- * HashAggregate (20) + : +- Exchange (19) + : +- * HashAggregate (18) + : +- * Project (17) + : +- * SortMergeJoin Inner (16) + : :- * Sort (9) + : : +- Exchange (8) + : : +- * Project (7) + : : +- * BroadcastHashJoin Inner BuildRight (6) + : : :- * Project (4) + : : : +- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet spark_catalog.default.web_sales (1) + : : +- ReusedExchange (5) + : +- * Sort (15) + : +- Exchange (14) + : +- * Project (13) + : +- * Filter (12) + : +- * ColumnarToRow (11) + : +- Scan parquet spark_catalog.default.web_returns (10) + :- * Project (54) + : +- * Filter (53) + : +- Window (52) + : +- * Sort (51) + : +- Window (50) + : +- * Sort (49) + : +- Exchange (48) + : +- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * SortMergeJoin Inner (43) + : :- * Sort (36) + : : +- Exchange (35) + : : +- * Project (34) + : : +- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Project (31) + : : : +- * Filter (30) + : : : +- * ColumnarToRow (29) + : : : +- Scan parquet spark_catalog.default.catalog_sales (28) + : : +- ReusedExchange (32) + : +- * Sort (42) + : +- Exchange (41) + : +- * Project (40) + : +- * Filter (39) + : +- * ColumnarToRow (38) + : +- Scan parquet spark_catalog.default.catalog_returns (37) + +- * Project (81) + +- * Filter (80) + +- Window (79) + +- * Sort (78) + +- Window (77) + +- * Sort (76) + +- Exchange (75) + +- * HashAggregate (74) + +- Exchange (73) + +- * HashAggregate (72) + +- * Project (71) + +- * SortMergeJoin Inner (70) + :- * Sort (63) + : +- Exchange (62) + : +- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * Project (58) + : : +- * Filter (57) + : : +- * ColumnarToRow (56) + : : +- Scan parquet spark_catalog.default.store_sales (55) + : +- ReusedExchange (59) + +- * Sort (69) + +- Exchange (68) + +- * Project (67) + +- * Filter (66) + +- * ColumnarToRow (65) + +- Scan parquet spark_catalog.default.store_returns (64) (1) Scan parquet spark_catalog.default.web_sales @@ -106,7 +105,7 @@ Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND Output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -(5) ReusedExchange [Reuses operator id: 91] +(5) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#8] (6) BroadcastHashJoin [codegen id : 2] @@ -228,7 +227,7 @@ Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AN Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -(32) ReusedExchange [Reuses operator id: 91] +(32) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#42] (33) BroadcastHashJoin [codegen id : 12] @@ -350,7 +349,7 @@ Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AN Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -(59) ReusedExchange [Reuses operator id: 91] +(59) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#76] (60) BroadcastHashJoin [codegen id : 22] @@ -462,52 +461,48 @@ Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(84) Exchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(85) HashAggregate [codegen id : 32] +(84) HashAggregate [codegen id : 31] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(86) TakeOrderedAndProject +(85) TakeOrderedAndProject Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (91) -+- * Project (90) - +- * Filter (89) - +- * ColumnarToRow (88) - +- Scan parquet spark_catalog.default.date_dim (87) +BroadcastExchange (90) ++- * Project (89) + +- * Filter (88) + +- * ColumnarToRow (87) + +- Scan parquet spark_catalog.default.date_dim (86) -(87) Scan parquet spark_catalog.default.date_dim +(86) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#8, d_year#104, d_moy#105] 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,12), IsNotNull(d_date_sk)] ReadSchema: struct -(88) ColumnarToRow [codegen id : 1] +(87) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#8, d_year#104, d_moy#105] -(89) Filter [codegen id : 1] +(88) Filter [codegen id : 1] Input [3]: [d_date_sk#8, d_year#104, d_moy#105] Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 = 2001)) AND (d_moy#105 = 12)) AND isnotnull(d_date_sk#8)) -(90) Project [codegen id : 1] +(89) Project [codegen id : 1] Output [1]: [d_date_sk#8] Input [3]: [d_date_sk#8, d_year#104, d_moy#105] -(91) BroadcastExchange +(90) BroadcastExchange Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/simplified.txt index 80d1661b033de..cc2ebb0c36ea8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/simplified.txt @@ -1,160 +1,157 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (32) + WholeStageCodegen (31) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (31) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (10) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (9) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (8) - Sort [return_ratio] - InputAdapter - Exchange #2 - WholeStageCodegen (7) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (6) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (10) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (9) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (8) + Sort [return_ratio] + InputAdapter + Exchange #1 + WholeStageCodegen (7) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ws_item_sk] #2 + WholeStageCodegen (6) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ws_order_number,ws_item_sk] InputAdapter - WholeStageCodegen (3) - Sort [ws_order_number,ws_item_sk] - InputAdapter - Exchange [ws_order_number,ws_item_sk] #4 - WholeStageCodegen (2) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + Exchange [ws_order_number,ws_item_sk] #3 + WholeStageCodegen (2) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #5 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + WholeStageCodegen (5) + Sort [wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (5) - Sort [wr_order_number,wr_item_sk] - InputAdapter - Exchange [wr_order_number,wr_item_sk] #6 - WholeStageCodegen (4) - Project [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - Filter [wr_return_amt,wr_order_number,wr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - WholeStageCodegen (20) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (19) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (18) - Sort [return_ratio] - InputAdapter - Exchange #7 - WholeStageCodegen (17) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #8 - WholeStageCodegen (16) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Exchange [wr_order_number,wr_item_sk] #5 + WholeStageCodegen (4) + Project [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + Filter [wr_return_amt,wr_order_number,wr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + WholeStageCodegen (20) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (19) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (18) + Sort [return_ratio] + InputAdapter + Exchange #6 + WholeStageCodegen (17) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #7 + WholeStageCodegen (16) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (13) + Sort [cs_order_number,cs_item_sk] InputAdapter - WholeStageCodegen (13) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #9 - WholeStageCodegen (12) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Exchange [cs_order_number,cs_item_sk] #8 + WholeStageCodegen (12) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #5 + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + WholeStageCodegen (15) + Sort [cr_order_number,cr_item_sk] InputAdapter - WholeStageCodegen (15) - Sort [cr_order_number,cr_item_sk] - InputAdapter - Exchange [cr_order_number,cr_item_sk] #10 - WholeStageCodegen (14) - Project [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - Filter [cr_return_amount,cr_order_number,cr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - WholeStageCodegen (30) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (29) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (28) - Sort [return_ratio] - InputAdapter - Exchange #11 - WholeStageCodegen (27) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ss_item_sk] #12 - WholeStageCodegen (26) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Exchange [cr_order_number,cr_item_sk] #9 + WholeStageCodegen (14) + Project [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + Filter [cr_return_amount,cr_order_number,cr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + WholeStageCodegen (30) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (29) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (28) + Sort [return_ratio] + InputAdapter + Exchange #10 + WholeStageCodegen (27) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ss_item_sk] #11 + WholeStageCodegen (26) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (23) + Sort [ss_ticket_number,ss_item_sk] InputAdapter - WholeStageCodegen (23) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #13 - WholeStageCodegen (22) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Exchange [ss_ticket_number,ss_item_sk] #12 + WholeStageCodegen (22) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #5 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + WholeStageCodegen (25) + Sort [sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (25) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #14 - WholeStageCodegen (24) - Project [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - Filter [sr_return_amt,sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + Exchange [sr_ticket_number,sr_item_sk] #13 + WholeStageCodegen (24) + Project [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + Filter [sr_return_amt,sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index fea7a9fe207df..5af506b8109b2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -1,81 +1,80 @@ == Physical Plan == -TakeOrderedAndProject (77) -+- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (24) - : +- * Filter (23) - : +- Window (22) - : +- * Sort (21) - : +- Window (20) - : +- * Sort (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- Exchange (16) - : +- * HashAggregate (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildLeft (10) - : : :- BroadcastExchange (5) - : : : +- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * Project (9) - : : +- * Filter (8) - : : +- * ColumnarToRow (7) - : : +- Scan parquet spark_catalog.default.web_returns (6) - : +- ReusedExchange (12) - :- * Project (48) - : +- * Filter (47) - : +- Window (46) - : +- * Sort (45) - : +- Window (44) - : +- * Sort (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildLeft (34) - : : :- BroadcastExchange (29) - : : : +- * Project (28) - : : : +- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet spark_catalog.default.catalog_sales (25) - : : +- * Project (33) - : : +- * Filter (32) - : : +- * ColumnarToRow (31) - : : +- Scan parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * Sort (67) - +- Exchange (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildLeft (58) - : :- BroadcastExchange (53) - : : +- * Project (52) - : : +- * Filter (51) - : : +- * ColumnarToRow (50) - : : +- Scan parquet spark_catalog.default.store_sales (49) - : +- * Project (57) - : +- * Filter (56) - : +- * ColumnarToRow (55) - : +- Scan parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (60) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (24) + : +- * Filter (23) + : +- Window (22) + : +- * Sort (21) + : +- Window (20) + : +- * Sort (19) + : +- Exchange (18) + : +- * HashAggregate (17) + : +- Exchange (16) + : +- * HashAggregate (15) + : +- * Project (14) + : +- * BroadcastHashJoin Inner BuildRight (13) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildLeft (10) + : : :- BroadcastExchange (5) + : : : +- * Project (4) + : : : +- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet spark_catalog.default.web_sales (1) + : : +- * Project (9) + : : +- * Filter (8) + : : +- * ColumnarToRow (7) + : : +- Scan parquet spark_catalog.default.web_returns (6) + : +- ReusedExchange (12) + :- * Project (48) + : +- * Filter (47) + : +- Window (46) + : +- * Sort (45) + : +- Window (44) + : +- * Sort (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (35) + : : +- * BroadcastHashJoin Inner BuildLeft (34) + : : :- BroadcastExchange (29) + : : : +- * Project (28) + : : : +- * Filter (27) + : : : +- * ColumnarToRow (26) + : : : +- Scan parquet spark_catalog.default.catalog_sales (25) + : : +- * Project (33) + : : +- * Filter (32) + : : +- * ColumnarToRow (31) + : : +- Scan parquet spark_catalog.default.catalog_returns (30) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * Sort (67) + +- Exchange (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * HashAggregate (63) + +- * Project (62) + +- * BroadcastHashJoin Inner BuildRight (61) + :- * Project (59) + : +- * BroadcastHashJoin Inner BuildLeft (58) + : :- BroadcastExchange (53) + : : +- * Project (52) + : : +- * Filter (51) + : : +- * ColumnarToRow (50) + : : +- Scan parquet spark_catalog.default.store_sales (49) + : +- * Project (57) + : +- * Filter (56) + : +- * ColumnarToRow (55) + : +- Scan parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (60) (1) Scan parquet spark_catalog.default.web_sales @@ -129,7 +128,7 @@ Join condition: None Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(12) ReusedExchange [Reuses operator id: 82] +(12) ReusedExchange [Reuses operator id: 81] Output [1]: [d_date_sk#13] (13) BroadcastHashJoin [codegen id : 3] @@ -239,7 +238,7 @@ Join condition: None Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(36) ReusedExchange [Reuses operator id: 82] +(36) ReusedExchange [Reuses operator id: 81] Output [1]: [d_date_sk#47] (37) BroadcastHashJoin [codegen id : 10] @@ -349,7 +348,7 @@ Join condition: None Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(60) ReusedExchange [Reuses operator id: 82] +(60) ReusedExchange [Reuses operator id: 81] Output [1]: [d_date_sk#81] (61) BroadcastHashJoin [codegen id : 17] @@ -417,52 +416,48 @@ Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(75) Exchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(76) HashAggregate [codegen id : 23] +(75) HashAggregate [codegen id : 22] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(77) TakeOrderedAndProject +(76) TakeOrderedAndProject Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (82) -+- * Project (81) - +- * Filter (80) - +- * ColumnarToRow (79) - +- Scan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (81) ++- * Project (80) + +- * Filter (79) + +- * ColumnarToRow (78) + +- Scan parquet spark_catalog.default.date_dim (77) -(78) Scan parquet spark_catalog.default.date_dim +(77) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#104, d_moy#105] 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,12), IsNotNull(d_date_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -(80) Filter [codegen id : 1] +(79) Filter [codegen id : 1] Input [3]: [d_date_sk#13, d_year#104, d_moy#105] Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 = 2001)) AND (d_moy#105 = 12)) AND isnotnull(d_date_sk#13)) -(81) Project [codegen id : 1] +(80) Project [codegen id : 1] Output [1]: [d_date_sk#13] Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -(82) BroadcastExchange +(81) BroadcastExchange Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index cfb4b948b4e8b..2ea310d01171d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -1,133 +1,130 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (23) + WholeStageCodegen (22) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - Sort [return_ratio] - InputAdapter - Exchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - Project [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - Filter [wr_return_amt,wr_order_number,wr_item_sk] + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (7) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (6) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (5) + Sort [return_ratio] + InputAdapter + Exchange #1 + WholeStageCodegen (4) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ws_item_sk] #2 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - Sort [return_ratio] - InputAdapter - Exchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + Project [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + Filter [wr_return_amt,wr_order_number,wr_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - Project [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - Filter [cr_return_amount,cr_order_number,cr_item_sk] + Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (14) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (13) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (12) + Sort [return_ratio] + InputAdapter + Exchange #5 + WholeStageCodegen (11) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #6 + WholeStageCodegen (10) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - Sort [return_ratio] - InputAdapter - Exchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + Project [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + Filter [cr_return_amount,cr_order_number,cr_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - Project [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - Filter [sr_return_amt,sr_ticket_number,sr_item_sk] + Scan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (21) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (20) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (19) + Sort [return_ratio] + InputAdapter + Exchange #8 + WholeStageCodegen (18) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ss_item_sk] #9 + WholeStageCodegen (17) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (15) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + Project [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + Filter [sr_return_amt,sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 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 3676f5f5fc21b..d088cb0190d7a 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 @@ -1,102 +1,133 @@ == Physical Plan == -TakeOrderedAndProject (98) -+- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- Union (94) - :- * HashAggregate (83) - : +- Exchange (82) - : +- * HashAggregate (81) - : +- Union (80) - : :- * Project (30) - : : +- * BroadcastHashJoin LeftOuter BuildRight (29) - : : :- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet spark_catalog.default.store (7) - : : +- BroadcastExchange (28) - : : +- * HashAggregate (27) - : : +- Exchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet spark_catalog.default.store_returns (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : :- * Project (49) - : : +- * BroadcastNestedLoopJoin Inner BuildRight (48) - : : :- * HashAggregate (38) - : : : +- Exchange (37) - : : : +- * HashAggregate (36) - : : : +- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (32) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (31) - : : : +- ReusedExchange (33) - : : +- BroadcastExchange (47) - : : +- * HashAggregate (46) - : : +- Exchange (45) - : : +- * HashAggregate (44) - : : +- * Project (43) - : : +- * BroadcastHashJoin Inner BuildRight (42) - : : :- * ColumnarToRow (40) - : : : +- Scan parquet spark_catalog.default.catalog_returns (39) - : : +- ReusedExchange (41) - : +- * Project (79) - : +- * BroadcastHashJoin LeftOuter BuildRight (78) - : :- * HashAggregate (64) - : : +- Exchange (63) - : : +- * HashAggregate (62) - : : +- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Filter (52) - : : : : +- * ColumnarToRow (51) - : : : : +- Scan parquet spark_catalog.default.web_sales (50) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * Filter (58) - : : +- * ColumnarToRow (57) - : : +- Scan parquet spark_catalog.default.web_page (56) - : +- BroadcastExchange (77) - : +- * HashAggregate (76) - : +- Exchange (75) - : +- * HashAggregate (74) - : +- * Project (73) - : +- * BroadcastHashJoin Inner BuildRight (72) - : :- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * Filter (67) - : : : +- * ColumnarToRow (66) - : : : +- Scan parquet spark_catalog.default.web_returns (65) - : : +- ReusedExchange (68) - : +- ReusedExchange (71) - :- * HashAggregate (88) - : +- Exchange (87) - : +- * HashAggregate (86) - : +- * HashAggregate (85) - : +- ReusedExchange (84) - +- * HashAggregate (93) - +- Exchange (92) - +- * HashAggregate (91) - +- * HashAggregate (90) - +- ReusedExchange (89) +TakeOrderedAndProject (129) ++- * HashAggregate (128) + +- Exchange (127) + +- * HashAggregate (126) + +- Union (125) + :- * HashAggregate (82) + : +- * HashAggregate (81) + : +- Union (80) + : :- * Project (30) + : : +- * BroadcastHashJoin LeftOuter BuildRight (29) + : : :- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet spark_catalog.default.store (7) + : : +- BroadcastExchange (28) + : : +- * HashAggregate (27) + : : +- Exchange (26) + : : +- * HashAggregate (25) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Filter (18) + : : : : +- * ColumnarToRow (17) + : : : : +- Scan parquet spark_catalog.default.store_returns (16) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : :- * Project (49) + : : +- * BroadcastNestedLoopJoin Inner BuildRight (48) + : : :- * HashAggregate (38) + : : : +- Exchange (37) + : : : +- * HashAggregate (36) + : : : +- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * ColumnarToRow (32) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (31) + : : : +- ReusedExchange (33) + : : +- BroadcastExchange (47) + : : +- * HashAggregate (46) + : : +- Exchange (45) + : : +- * HashAggregate (44) + : : +- * Project (43) + : : +- * BroadcastHashJoin Inner BuildRight (42) + : : :- * ColumnarToRow (40) + : : : +- Scan parquet spark_catalog.default.catalog_returns (39) + : : +- ReusedExchange (41) + : +- * Project (79) + : +- * BroadcastHashJoin LeftOuter BuildRight (78) + : :- * HashAggregate (64) + : : +- Exchange (63) + : : +- * HashAggregate (62) + : : +- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Filter (52) + : : : : +- * ColumnarToRow (51) + : : : : +- Scan parquet spark_catalog.default.web_sales (50) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (59) + : : +- * Filter (58) + : : +- * ColumnarToRow (57) + : : +- Scan parquet spark_catalog.default.web_page (56) + : +- BroadcastExchange (77) + : +- * HashAggregate (76) + : +- Exchange (75) + : +- * HashAggregate (74) + : +- * Project (73) + : +- * BroadcastHashJoin Inner BuildRight (72) + : :- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- * Filter (67) + : : : +- * ColumnarToRow (66) + : : : +- Scan parquet spark_catalog.default.web_returns (65) + : : +- ReusedExchange (68) + : +- ReusedExchange (71) + :- * HashAggregate (103) + : +- Exchange (102) + : +- * HashAggregate (101) + : +- * HashAggregate (100) + : +- * HashAggregate (99) + : +- Union (98) + : :- * Project (87) + : : +- * BroadcastHashJoin LeftOuter BuildRight (86) + : : :- * HashAggregate (84) + : : : +- ReusedExchange (83) + : : +- ReusedExchange (85) + : :- * Project (92) + : : +- * BroadcastNestedLoopJoin Inner BuildRight (91) + : : :- * HashAggregate (89) + : : : +- ReusedExchange (88) + : : +- ReusedExchange (90) + : +- * Project (97) + : +- * BroadcastHashJoin LeftOuter BuildRight (96) + : :- * HashAggregate (94) + : : +- ReusedExchange (93) + : +- ReusedExchange (95) + +- * HashAggregate (124) + +- Exchange (123) + +- * HashAggregate (122) + +- * HashAggregate (121) + +- * HashAggregate (120) + +- Union (119) + :- * Project (108) + : +- * BroadcastHashJoin LeftOuter BuildRight (107) + : :- * HashAggregate (105) + : : +- ReusedExchange (104) + : +- ReusedExchange (106) + :- * Project (113) + : +- * BroadcastNestedLoopJoin Inner BuildRight (112) + : :- * HashAggregate (110) + : : +- ReusedExchange (109) + : +- ReusedExchange (111) + +- * Project (118) + +- * BroadcastHashJoin LeftOuter BuildRight (117) + :- * HashAggregate (115) + : +- ReusedExchange (114) + +- ReusedExchange (116) (1) Scan parquet spark_catalog.default.store_sales @@ -114,7 +145,7 @@ Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_s Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Condition : isnotnull(ss_store_sk#1) -(4) ReusedExchange [Reuses operator id: 103] +(4) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -188,7 +219,7 @@ Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_s Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] Condition : isnotnull(sr_store_sk#16) -(19) ReusedExchange [Reuses operator id: 103] +(19) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#20] (20) BroadcastHashJoin [codegen id : 6] @@ -256,7 +287,7 @@ ReadSchema: struct (40) ColumnarToRow [codegen id : 12] Input [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] -(41) ReusedExchange [Reuses operator id: 103] +(41) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#50] (42) BroadcastHashJoin [codegen id : 12] @@ -355,7 +386,7 @@ Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_ Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] Condition : isnotnull(ws_web_page_sk#62) -(53) ReusedExchange [Reuses operator id: 103] +(53) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#66] (54) BroadcastHashJoin [codegen id : 17] @@ -429,7 +460,7 @@ Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_dat Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] Condition : isnotnull(wr_web_page_sk#76) -(68) ReusedExchange [Reuses operator id: 103] +(68) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#80] (69) BroadcastHashJoin [codegen id : 20] @@ -496,128 +527,270 @@ Functions [3]: [partial_sum(sales#14), partial_sum(returns#32), partial_sum(prof Aggregate Attributes [6]: [sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] Results [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -(82) Exchange -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Arguments: hashpartitioning(channel#30, id#31, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(83) HashAggregate [codegen id : 24] +(82) HashAggregate [codegen id : 23] 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 [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#112, id#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] - -(85) HashAggregate [codegen id : 48] -Input [8]: [channel#112, id#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Keys [2]: [channel#112, id#113] -Functions [3]: [sum(sales#120), sum(returns#121), sum(profit#122)] -Aggregate Attributes [3]: [sum(sales#120)#106, sum(returns#121)#107, sum(profit#122)#108] -Results [4]: [channel#112, sum(sales#120)#106 AS sales#123, sum(returns#121)#107 AS returns#124, sum(profit#122)#108 AS profit#125] - -(86) HashAggregate [codegen id : 48] -Input [4]: [channel#112, sales#123, returns#124, profit#125] -Keys [1]: [channel#112] -Functions [3]: [partial_sum(sales#123), partial_sum(returns#124), partial_sum(profit#125)] -Aggregate Attributes [6]: [sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Results [7]: [channel#112, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] - -(87) Exchange -Input [7]: [channel#112, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] -Arguments: hashpartitioning(channel#112, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(88) HashAggregate [codegen id : 49] -Input [7]: [channel#112, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] -Keys [1]: [channel#112] -Functions [3]: [sum(sales#123), sum(returns#124), sum(profit#125)] -Aggregate Attributes [3]: [sum(sales#123)#138, sum(returns#124)#139, sum(profit#125)#140] -Results [5]: [channel#112, null AS id#141, sum(sales#123)#138 AS sales#142, sum(returns#124)#139 AS returns#143, sum(profit#125)#140 AS profit#144] - -(89) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] - -(90) HashAggregate [codegen id : 73] -Input [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -Keys [2]: [channel#145, id#146] -Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] -Aggregate Attributes [3]: [sum(sales#153)#106, sum(returns#154)#107, sum(profit#155)#108] -Results [3]: [sum(sales#153)#106 AS sales#156, sum(returns#154)#107 AS returns#157, sum(profit#155)#108 AS profit#158] - -(91) HashAggregate [codegen id : 73] -Input [3]: [sales#156, returns#157, profit#158] +(83) ReusedExchange [Reuses operator id: 14] +Output [3]: [s_store_sk#112, sum#113, sum#114] + +(84) HashAggregate [codegen id : 31] +Input [3]: [s_store_sk#112, sum#113, sum#114] +Keys [1]: [s_store_sk#112] +Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#115)), sum(UnscaledValue(ss_net_profit#116))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#115))#12, sum(UnscaledValue(ss_net_profit#116))#13] +Results [3]: [s_store_sk#112, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#115))#12,17,2) AS sales#117, MakeDecimal(sum(UnscaledValue(ss_net_profit#116))#13,17,2) AS profit#118] + +(85) ReusedExchange [Reuses operator id: 28] +Output [3]: [s_store_sk#119, returns#120, profit_loss#121] + +(86) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [s_store_sk#112] +Right keys [1]: [s_store_sk#119] +Join type: LeftOuter +Join condition: None + +(87) Project [codegen id : 31] +Output [5]: [store channel AS channel#122, s_store_sk#112 AS id#123, sales#117, coalesce(returns#120, 0.00) AS returns#124, (profit#118 - coalesce(profit_loss#121, 0.00)) AS profit#125] +Input [6]: [s_store_sk#112, sales#117, profit#118, s_store_sk#119, returns#120, profit_loss#121] + +(88) ReusedExchange [Reuses operator id: 37] +Output [3]: [cs_call_center_sk#126, sum#127, sum#128] + +(89) HashAggregate [codegen id : 37] +Input [3]: [cs_call_center_sk#126, sum#127, sum#128] +Keys [1]: [cs_call_center_sk#126] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#129)), sum(UnscaledValue(cs_net_profit#130))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#129))#43, sum(UnscaledValue(cs_net_profit#130))#44] +Results [3]: [cs_call_center_sk#126, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#129))#43,17,2) AS sales#131, MakeDecimal(sum(UnscaledValue(cs_net_profit#130))#44,17,2) AS profit#132] + +(90) ReusedExchange [Reuses operator id: 47] +Output [2]: [returns#133, profit_loss#134] + +(91) BroadcastNestedLoopJoin [codegen id : 37] +Join type: Inner +Join condition: None + +(92) Project [codegen id : 37] +Output [5]: [catalog channel AS channel#135, cs_call_center_sk#126 AS id#136, sales#131, returns#133, (profit#132 - profit_loss#134) AS profit#137] +Input [5]: [cs_call_center_sk#126, sales#131, profit#132, returns#133, profit_loss#134] + +(93) ReusedExchange [Reuses operator id: 63] +Output [3]: [wp_web_page_sk#138, sum#139, sum#140] + +(94) HashAggregate [codegen id : 45] +Input [3]: [wp_web_page_sk#138, sum#139, sum#140] +Keys [1]: [wp_web_page_sk#138] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#141)), sum(UnscaledValue(ws_net_profit#142))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#141))#72, sum(UnscaledValue(ws_net_profit#142))#73] +Results [3]: [wp_web_page_sk#138, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#141))#72,17,2) AS sales#143, MakeDecimal(sum(UnscaledValue(ws_net_profit#142))#73,17,2) AS profit#144] + +(95) ReusedExchange [Reuses operator id: 77] +Output [3]: [wp_web_page_sk#145, returns#146, profit_loss#147] + +(96) BroadcastHashJoin [codegen id : 45] +Left keys [1]: [wp_web_page_sk#138] +Right keys [1]: [wp_web_page_sk#145] +Join type: LeftOuter +Join condition: None + +(97) Project [codegen id : 45] +Output [5]: [web channel AS channel#148, wp_web_page_sk#138 AS id#149, sales#143, coalesce(returns#146, 0.00) AS returns#150, (profit#144 - coalesce(profit_loss#147, 0.00)) AS profit#151] +Input [6]: [wp_web_page_sk#138, sales#143, profit#144, wp_web_page_sk#145, returns#146, profit_loss#147] + +(98) Union + +(99) HashAggregate [codegen id : 46] +Input [5]: [channel#122, id#123, sales#117, returns#124, profit#125] +Keys [2]: [channel#122, id#123] +Functions [3]: [partial_sum(sales#117), partial_sum(returns#124), partial_sum(profit#125)] +Aggregate Attributes [6]: [sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] +Results [8]: [channel#122, id#123, sum#158, isEmpty#159, sum#160, isEmpty#161, sum#162, isEmpty#163] + +(100) HashAggregate [codegen id : 46] +Input [8]: [channel#122, id#123, sum#158, isEmpty#159, sum#160, isEmpty#161, sum#162, isEmpty#163] +Keys [2]: [channel#122, id#123] +Functions [3]: [sum(sales#117), sum(returns#124), sum(profit#125)] +Aggregate Attributes [3]: [sum(sales#117)#106, sum(returns#124)#107, sum(profit#125)#108] +Results [4]: [channel#122, sum(sales#117)#106 AS sales#164, sum(returns#124)#107 AS returns#165, sum(profit#125)#108 AS profit#166] + +(101) HashAggregate [codegen id : 46] +Input [4]: [channel#122, sales#164, returns#165, profit#166] +Keys [1]: [channel#122] +Functions [3]: [partial_sum(sales#164), partial_sum(returns#165), partial_sum(profit#166)] +Aggregate Attributes [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] +Results [7]: [channel#122, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] + +(102) Exchange +Input [7]: [channel#122, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] +Arguments: hashpartitioning(channel#122, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(103) HashAggregate [codegen id : 47] +Input [7]: [channel#122, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] +Keys [1]: [channel#122] +Functions [3]: [sum(sales#164), sum(returns#165), sum(profit#166)] +Aggregate Attributes [3]: [sum(sales#164)#179, sum(returns#165)#180, sum(profit#166)#181] +Results [5]: [channel#122, null AS id#182, sum(sales#164)#179 AS sales#183, sum(returns#165)#180 AS returns#184, sum(profit#166)#181 AS profit#185] + +(104) ReusedExchange [Reuses operator id: 14] +Output [3]: [s_store_sk#186, sum#187, sum#188] + +(105) HashAggregate [codegen id : 55] +Input [3]: [s_store_sk#186, sum#187, sum#188] +Keys [1]: [s_store_sk#186] +Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#189)), sum(UnscaledValue(ss_net_profit#190))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#189))#12, sum(UnscaledValue(ss_net_profit#190))#13] +Results [3]: [s_store_sk#186, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#189))#12,17,2) AS sales#191, MakeDecimal(sum(UnscaledValue(ss_net_profit#190))#13,17,2) AS profit#192] + +(106) ReusedExchange [Reuses operator id: 28] +Output [3]: [s_store_sk#193, returns#194, profit_loss#195] + +(107) BroadcastHashJoin [codegen id : 55] +Left keys [1]: [s_store_sk#186] +Right keys [1]: [s_store_sk#193] +Join type: LeftOuter +Join condition: None + +(108) Project [codegen id : 55] +Output [5]: [store channel AS channel#196, s_store_sk#186 AS id#197, sales#191, coalesce(returns#194, 0.00) AS returns#198, (profit#192 - coalesce(profit_loss#195, 0.00)) AS profit#199] +Input [6]: [s_store_sk#186, sales#191, profit#192, s_store_sk#193, returns#194, profit_loss#195] + +(109) ReusedExchange [Reuses operator id: 37] +Output [3]: [cs_call_center_sk#200, sum#201, sum#202] + +(110) HashAggregate [codegen id : 61] +Input [3]: [cs_call_center_sk#200, sum#201, sum#202] +Keys [1]: [cs_call_center_sk#200] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#203)), sum(UnscaledValue(cs_net_profit#204))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#203))#43, sum(UnscaledValue(cs_net_profit#204))#44] +Results [3]: [cs_call_center_sk#200, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#203))#43,17,2) AS sales#205, MakeDecimal(sum(UnscaledValue(cs_net_profit#204))#44,17,2) AS profit#206] + +(111) ReusedExchange [Reuses operator id: 47] +Output [2]: [returns#207, profit_loss#208] + +(112) BroadcastNestedLoopJoin [codegen id : 61] +Join type: Inner +Join condition: None + +(113) Project [codegen id : 61] +Output [5]: [catalog channel AS channel#209, cs_call_center_sk#200 AS id#210, sales#205, returns#207, (profit#206 - profit_loss#208) AS profit#211] +Input [5]: [cs_call_center_sk#200, sales#205, profit#206, returns#207, profit_loss#208] + +(114) ReusedExchange [Reuses operator id: 63] +Output [3]: [wp_web_page_sk#212, sum#213, sum#214] + +(115) HashAggregate [codegen id : 69] +Input [3]: [wp_web_page_sk#212, sum#213, sum#214] +Keys [1]: [wp_web_page_sk#212] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#215)), sum(UnscaledValue(ws_net_profit#216))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#215))#72, sum(UnscaledValue(ws_net_profit#216))#73] +Results [3]: [wp_web_page_sk#212, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#215))#72,17,2) AS sales#217, MakeDecimal(sum(UnscaledValue(ws_net_profit#216))#73,17,2) AS profit#218] + +(116) ReusedExchange [Reuses operator id: 77] +Output [3]: [wp_web_page_sk#219, returns#220, profit_loss#221] + +(117) BroadcastHashJoin [codegen id : 69] +Left keys [1]: [wp_web_page_sk#212] +Right keys [1]: [wp_web_page_sk#219] +Join type: LeftOuter +Join condition: None + +(118) Project [codegen id : 69] +Output [5]: [web channel AS channel#222, wp_web_page_sk#212 AS id#223, sales#217, coalesce(returns#220, 0.00) AS returns#224, (profit#218 - coalesce(profit_loss#221, 0.00)) AS profit#225] +Input [6]: [wp_web_page_sk#212, sales#217, profit#218, wp_web_page_sk#219, returns#220, profit_loss#221] + +(119) Union + +(120) HashAggregate [codegen id : 70] +Input [5]: [channel#196, id#197, sales#191, returns#198, profit#199] +Keys [2]: [channel#196, id#197] +Functions [3]: [partial_sum(sales#191), partial_sum(returns#198), partial_sum(profit#199)] +Aggregate Attributes [6]: [sum#226, isEmpty#227, sum#228, isEmpty#229, sum#230, isEmpty#231] +Results [8]: [channel#196, id#197, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237] + +(121) HashAggregate [codegen id : 70] +Input [8]: [channel#196, id#197, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237] +Keys [2]: [channel#196, id#197] +Functions [3]: [sum(sales#191), sum(returns#198), sum(profit#199)] +Aggregate Attributes [3]: [sum(sales#191)#106, sum(returns#198)#107, sum(profit#199)#108] +Results [3]: [sum(sales#191)#106 AS sales#238, sum(returns#198)#107 AS returns#239, sum(profit#199)#108 AS profit#240] + +(122) HashAggregate [codegen id : 70] +Input [3]: [sales#238, returns#239, profit#240] Keys: [] -Functions [3]: [partial_sum(sales#156), partial_sum(returns#157), partial_sum(profit#158)] -Aggregate Attributes [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Results [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] +Functions [3]: [partial_sum(sales#238), partial_sum(returns#239), partial_sum(profit#240)] +Aggregate Attributes [6]: [sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246] +Results [6]: [sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252] -(92) Exchange -Input [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] +(123) Exchange +Input [6]: [sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(93) HashAggregate [codegen id : 74] -Input [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] +(124) HashAggregate [codegen id : 71] +Input [6]: [sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252] Keys: [] -Functions [3]: [sum(sales#156), sum(returns#157), sum(profit#158)] -Aggregate Attributes [3]: [sum(sales#156)#171, sum(returns#157)#172, sum(profit#158)#173] -Results [5]: [null AS channel#174, null AS id#175, sum(sales#156)#171 AS sales#176, sum(returns#157)#172 AS returns#177, sum(profit#158)#173 AS profit#178] +Functions [3]: [sum(sales#238), sum(returns#239), sum(profit#240)] +Aggregate Attributes [3]: [sum(sales#238)#253, sum(returns#239)#254, sum(profit#240)#255] +Results [5]: [null AS channel#256, null AS id#257, sum(sales#238)#253 AS sales#258, sum(returns#239)#254 AS returns#259, sum(profit#240)#255 AS profit#260] -(94) Union +(125) Union -(95) HashAggregate [codegen id : 75] +(126) HashAggregate [codegen id : 72] Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Keys [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Functions: [] Aggregate Attributes: [] Results [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -(96) Exchange +(127) Exchange Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Arguments: hashpartitioning(channel#30, id#31, sales#109, returns#110, profit#111, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: hashpartitioning(channel#30, id#31, sales#109, returns#110, profit#111, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(97) HashAggregate [codegen id : 76] +(128) HashAggregate [codegen id : 73] Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Keys [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Functions: [] Aggregate Attributes: [] Results [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -(98) TakeOrderedAndProject +(129) TakeOrderedAndProject Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Arguments: 100, [channel#30 ASC NULLS FIRST, id#31 ASC NULLS FIRST], [channel#30, id#31, sales#109, returns#110, profit#111] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (103) -+- * Project (102) - +- * Filter (101) - +- * ColumnarToRow (100) - +- Scan parquet spark_catalog.default.date_dim (99) +BroadcastExchange (134) ++- * Project (133) + +- * Filter (132) + +- * ColumnarToRow (131) + +- Scan parquet spark_catalog.default.date_dim (130) -(99) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#179] +(130) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_date#261] 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#179] +(131) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_date#261] -(101) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_date#179] -Condition : (((isnotnull(d_date#179) AND (d_date#179 >= 1998-08-04)) AND (d_date#179 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) +(132) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_date#261] +Condition : (((isnotnull(d_date#261) AND (d_date#261 >= 1998-08-04)) AND (d_date#261 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(102) Project [codegen id : 1] +(133) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_date#179] +Input [2]: [d_date_sk#6, d_date#261] -(103) BroadcastExchange +(134) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] Subquery:2 Hosting operator id = 16 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/simplified.txt index 567e325d1a544..e82c8494b698d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/simplified.txt @@ -1,168 +1,215 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (76) + WholeStageCodegen (73) HashAggregate [channel,id,sales,returns,profit] InputAdapter Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (75) + WholeStageCodegen (72) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (24) + WholeStageCodegen (23) HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [channel,id] #2 - WholeStageCodegen (23) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (8) - Project [s_store_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (8) + Project [s_store_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [s_store_sk,s_store_sk] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + InputAdapter + Exchange [s_store_sk] #2 + WholeStageCodegen (3) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange [s_store_sk] #3 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] + Exchange [s_store_sk] #6 + WholeStageCodegen (6) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_store_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] + ReusedExchange [s_store_sk] #4 + WholeStageCodegen (14) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] + InputAdapter + Exchange [cs_call_center_sk] #7 + WholeStageCodegen (10) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - Exchange [s_store_sk] #7 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (13) + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange [cs_call_center_sk] #8 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Exchange #9 + WholeStageCodegen (12) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (13) - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (22) + Project [wp_web_page_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + InputAdapter + Exchange [wp_web_page_sk] #10 + WholeStageCodegen (17) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_web_page_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - Exchange #10 - WholeStageCodegen (12) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) - Project [wp_web_page_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + BroadcastExchange #11 + WholeStageCodegen (16) + Filter [wp_web_page_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (21) + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange [wp_web_page_sk] #11 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_web_page_sk] + Exchange [wp_web_page_sk] #13 + WholeStageCodegen (20) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + Filter [wr_web_page_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + Scan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - Filter [wp_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - InputAdapter - Exchange [wp_web_page_sk] #14 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [wp_web_page_sk] #12 - WholeStageCodegen (49) + ReusedExchange [wp_web_page_sk] #11 + WholeStageCodegen (47) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [channel] #15 - WholeStageCodegen (48) + Exchange [channel] #14 + WholeStageCodegen (46) HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (31) + Project [s_store_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [s_store_sk,s_store_sk] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + InputAdapter + ReusedExchange [s_store_sk,sum,sum] #2 + InputAdapter + ReusedExchange [s_store_sk,returns,profit_loss] #5 + WholeStageCodegen (37) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] + InputAdapter + ReusedExchange [cs_call_center_sk,sum,sum] #7 + InputAdapter + ReusedExchange [returns,profit_loss] #8 + WholeStageCodegen (45) + Project [wp_web_page_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + InputAdapter + ReusedExchange [wp_web_page_sk,sum,sum] #10 + InputAdapter + ReusedExchange [wp_web_page_sk,returns,profit_loss] #12 + WholeStageCodegen (71) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange #16 - WholeStageCodegen (73) + Exchange #15 + WholeStageCodegen (70) HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (55) + Project [s_store_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [s_store_sk,s_store_sk] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + InputAdapter + ReusedExchange [s_store_sk,sum,sum] #2 + InputAdapter + ReusedExchange [s_store_sk,returns,profit_loss] #5 + WholeStageCodegen (61) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] + InputAdapter + ReusedExchange [cs_call_center_sk,sum,sum] #7 + InputAdapter + ReusedExchange [returns,profit_loss] #8 + WholeStageCodegen (69) + Project [wp_web_page_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + InputAdapter + ReusedExchange [wp_web_page_sk,sum,sum] #10 + InputAdapter + ReusedExchange [wp_web_page_sk,returns,profit_loss] #12 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala index 5ddbb40fff1d5..e778690076ea5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala @@ -22,9 +22,11 @@ import java.util.Locale import org.apache.spark.sql.catalyst.optimizer.RemoveNoopUnion import org.apache.spark.sql.catalyst.plans.logical.Union +import org.apache.spark.sql.catalyst.plans.physical.UnknownPartitioning import org.apache.spark.sql.execution.{SparkPlan, UnionExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession, SQLTestData} @@ -1508,6 +1510,113 @@ class DataFrameSetOperationsSuite extends QueryTest } } } + + test("SPARK-52921: union partitioning - reused shuffle") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val df1 = Seq((1, 2, 4), (1, 3, 5), (2, 2, 3), (2, 4, 5)).toDF("a", "b", "c") + val df2 = Seq((1, 2, 4), (1, 3, 5), (2, 2, 3), (2, 4, 5)).toDF("a", "b", "c") + + val union = df1.repartition($"a").union(df2.repartition($"a")) + val unionExec = union.queryExecution.executedPlan.collect { + case u: UnionExec => u + } + assert(unionExec.size == 1) + + val shuffle = df1.repartition($"a").queryExecution.executedPlan.collect { + case s: ShuffleExchangeExec => s + } + assert(shuffle.size == 1) + + val reuseShuffle = union.queryExecution.executedPlan.collect { + case r: ReusedExchangeExec => r + } + assert(reuseShuffle.size == 1) + + val childPartitioning = shuffle.head.outputPartitioning + val partitioning = unionExec.head.outputPartitioning + assert(partitioning == childPartitioning) + } + } + + test("SPARK-52921: union partitioning - semantic equality") { + val df1 = Seq((1, 2, 4), (1, 3, 5), (2, 2, 3), (2, 4, 5)).toDF("a", "b", "c") + val df2 = Seq((4, 1, 5), (2, 4, 6), (1, 4, 2), (3, 5, 1)).toDF("d", "e", "f") + + val correctResult = withSQLConf(SQLConf.UNION_OUTPUT_PARTITIONING.key -> "false") { + df1.repartition($"a").union(df2.repartition($"d")).collect() + } + + Seq(true, false).foreach { enabled => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.UNION_OUTPUT_PARTITIONING.key -> enabled.toString) { + + val union = df1.repartition($"a").union(df2.repartition($"d")) + val unionExec = union.queryExecution.executedPlan.collect { + case u: UnionExec => u + } + assert(unionExec.size == 1) + + val shuffle = df1.repartition($"a").queryExecution.executedPlan.collect { + case s: ShuffleExchangeExec => s + } + assert(shuffle.size == 1) + + val childPartitioning = shuffle.head.outputPartitioning + val partitioning = unionExec.head.outputPartitioning + if (enabled) { + assert(partitioning == childPartitioning) + } + + checkAnswer(union, correctResult) + + // Avoid unnecessary shuffle if union output partitioning is enabled + val shuffledUnion = union.repartition($"a") + val shuffleNumBefore = union.queryExecution.executedPlan.collect { + case s: ShuffleExchangeExec => s + } + val shuffleNumAfter = shuffledUnion.queryExecution.executedPlan.collect { + case s: ShuffleExchangeExec => s + } + + if (enabled) { + assert(shuffleNumBefore.size == shuffleNumAfter.size) + } else { + assert(shuffleNumBefore.size + 1 == shuffleNumAfter.size) + } + checkAnswer(union, shuffledUnion) + } + } + } + + test("SPARK-52921: union partitioning - range partitioning") { + val df1 = Seq((1, 2, 4), (1, 3, 5), (2, 2, 3), (2, 4, 5)).toDF("a", "b", "c") + val df2 = Seq((4, 1, 5), (2, 4, 6), (1, 4, 2), (3, 5, 1)).toDF("d", "e", "f") + + val correctResult = withSQLConf(SQLConf.UNION_OUTPUT_PARTITIONING.key -> "false") { + df1.repartitionByRange($"a").union(df2.repartitionByRange($"d")).collect() + } + + Seq(true, false).foreach { enabled => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.UNION_OUTPUT_PARTITIONING.key -> enabled.toString) { + + val union = df1.repartitionByRange($"a").union(df2.repartitionByRange($"d")) + val unionExec = union.queryExecution.executedPlan.collect { + case u: UnionExec => u + } + assert(unionExec.size == 1) + + // For range partitioning, even children have the same partitioning, + // the union output partitioning is still UnknownPartitioning. + val partitioning = unionExec.head.outputPartitioning + assert(partitioning.isInstanceOf[UnknownPartitioning]) + + checkAnswer(union, correctResult) + } + } + } } case class UnionClass1a(a: Int, b: Long, nested: UnionClass2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 05c1012200dfa..07d346167baa4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -2089,7 +2089,15 @@ class AdaptiveQueryExecSuite """.stripMargin), numUnion = if (combineUnionEnabled) 1 else 2, numShuffleReader = 3, - numPartition = 1 + 1 + 2) + // SPARK-52921 + // If `combineUnionEnabled` is false, there are 2 unions. + // The inner union has 1 partition because its children have the same partitioning: + // CoalescedHashPartitioning(HashPartitioning(key, 10), CoalescedBoundary(0,10)). + // The outer union has 1 (inner union) + 2 (t1) partitions. + // + // If `combineUnionEnabled` is true, there is only 1 union. As the children have + // different partitioning, the union will have sum of children partitions. + numPartition = if (combineUnionEnabled) 1 + 1 + 2 else 1 + 2) // negative test checkResultPartition( @@ -3196,6 +3204,82 @@ class AdaptiveQueryExecSuite } } } + + test("SPARK-52921: Specify outputPartitioning for UnionExec for same output partitoning") { + def checkResultPartition( + df: Dataset[Row], + numUnion: Int, + numShuffleReader: Int, + numPartition: Int): Unit = { + df.collect() + assert(collect(df.queryExecution.executedPlan) { + case u: UnionExec => u + }.size == numUnion) + assert(collect(df.queryExecution.executedPlan) { + case r: AQEShuffleReadExec => r + }.size === numShuffleReader) + assert(df.rdd.partitions.length === numPartition) + } + + Seq(true, false).foreach { combineUnionEnabled => + val combineUnionConfig = if (combineUnionEnabled) { + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> "" + } else { + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> + "org.apache.spark.sql.catalyst.optimizer.CombineUnions" + } + // advisory partition size 1048576 has no special meaning, just a big enough value + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "1048576", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "10", + combineUnionConfig) { + withTempView("t1", "t2") { + spark.sparkContext.parallelize((1 to 10).map(i => TestData(i, i.toString)), 2) + .toDF().createOrReplaceTempView("t1") + spark.sparkContext.parallelize((1 to 10).map(i => TestData(i, i.toString)), 4) + .toDF().createOrReplaceTempView("t2") + + val query = + """ + |SELECT /*+ merge(t2) */ t1.key, t2.key FROM t1 JOIN t2 ON t1.key = t2.key + |UNION ALL + |SELECT key, count(*) FROM t2 GROUP BY key + |UNION ALL + |SELECT * FROM t1 + |""".stripMargin + + val correctResults = withSQLConf(SQLConf.UNION_OUTPUT_PARTITIONING.key -> "false") { + checkResultPartition( + sql(query), + numUnion = if (combineUnionEnabled) 1 else 2, + numShuffleReader = 3, + numPartition = 1 + 1 + 2) + + sql(query).collect() + } + + withSQLConf(SQLConf.UNION_OUTPUT_PARTITIONING.key -> "true") { + checkResultPartition( + sql(query), + numUnion = if (combineUnionEnabled) 1 else 2, + numShuffleReader = 3, + // If `combineUnionEnabled` is false, there are 2 unions. + // The inner union has 1 partition because its children have the same partitioning: + // CoalescedHashPartitioning(HashPartitioning(key, 10), CoalescedBoundary(0,10)). + // The outer union has 1 (inner union) + 2 (t1) partitions. + // + // If `combineUnionEnabled` is true, there is only 1 union. As the children have + // different partitioning, the union will have sum of children partitions. + numPartition = if (combineUnionEnabled) 1 + 1 + 2 else 1 + 2) + + checkAnswer(sql(query), correctResults) + } + } + } + } + } } /**