You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by do...@apache.org on 2023/02/17 04:48:15 UTC
[spark] branch master updated: [SPARK-39851][SQL] Improve join stats estimation if one side can keep uniqueness
This is an automated email from the ASF dual-hosted git repository.
dongjoon pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push:
new 1c0bd1f9f81 [SPARK-39851][SQL] Improve join stats estimation if one side can keep uniqueness
1c0bd1f9f81 is described below
commit 1c0bd1f9f813a341bbfdecb2c5ccde7fbc1bac2d
Author: wankunde <wa...@163.com>
AuthorDate: Thu Feb 16 20:48:02 2023 -0800
[SPARK-39851][SQL] Improve join stats estimation if one side can keep uniqueness
### What changes were proposed in this pull request?
This PR improves join stats estimation if one side can keep uniqueness(The distinct keys of the children of the join are a subset of the join keys). A common case is:
```sql
SELECT i_item_sk ss_item_sk
FROM item,
(SELECT DISTINCT iss.i_brand_id brand_id,
iss.i_class_id class_id,
iss.i_category_id category_id
FROM item iss) x
WHERE i_brand_id = brand_id
AND i_class_id = class_id
AND i_category_id = category_id
```
In this case, the row count of the join will definitely not expand.
Before this PR:
```
== Optimized Logical Plan ==
Project [i_item_sk#4 AS ss_item_sk#54], Statistics(sizeInBytes=370.8 MiB, rowCount=3.24E+7)
+- Join Inner, (((i_brand_id#11 = brand_id#51) AND (i_class_id#13 = class_id#52)) AND (i_category_id#15 = category_id#53)), Statistics(sizeInBytes=1112.3 MiB, rowCount=3.24E+7)
:- Project [i_item_sk#4, i_brand_id#11, i_class_id#13, i_category_id#15], Statistics(sizeInBytes=4.6 MiB, rowCount=2.02E+5)
: +- Filter ((isnotnull(i_brand_id#11) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#15)), Statistics(sizeInBytes=84.6 MiB, rowCount=2.02E+5)
: +- Relation spark_catalog.default.item[i_item_sk#4,i_item_id#5,i_rec_start_date#6,i_rec_end_date#7,i_item_desc#8,i_current_price#9,i_wholesale_cost#10,i_brand_id#11,i_brand#12,i_class_id#13,i_class#14,i_category_id#15,i_category#16,i_manufact_id#17,i_manufact#18,i_size#19,i_formulation#20,i_color#21,i_units#22,i_container#23,i_manager_id#24,i_product_name#25] parquet, Statistics(sizeInBytes=85.2 MiB, rowCount=2.04E+5)
+- Aggregate [brand_id#51, class_id#52, category_id#53], [brand_id#51, class_id#52, category_id#53], Statistics(sizeInBytes=2.6 MiB, rowCount=1.37E+5)
+- Project [i_brand_id#62 AS brand_id#51, i_class_id#64 AS class_id#52, i_category_id#66 AS category_id#53], Statistics(sizeInBytes=3.9 MiB, rowCount=2.02E+5)
+- Filter ((isnotnull(i_brand_id#62) AND isnotnull(i_class_id#64)) AND isnotnull(i_category_id#66)), Statistics(sizeInBytes=84.6 MiB, rowCount=2.02E+5)
+- Relation spark_catalog.default.item[i_item_sk#55,i_item_id#56,i_rec_start_date#57,i_rec_end_date#58,i_item_desc#59,i_current_price#60,i_wholesale_cost#61,i_brand_id#62,i_brand#63,i_class_id#64,i_class#65,i_category_id#66,i_category#67,i_manufact_id#68,i_manufact#69,i_size#70,i_formulation#71,i_color#72,i_units#73,i_container#74,i_manager_id#75,i_product_name#76] parquet, Statistics(sizeInBytes=85.2 MiB, rowCount=2.04E+5)
```
After this PR:
```
== Optimized Logical Plan ==
Project [i_item_sk#4 AS ss_item_sk#54], Statistics(sizeInBytes=2.3 MiB, rowCount=2.02E+5)
+- Join Inner, (((i_brand_id#11 = brand_id#51) AND (i_class_id#13 = class_id#52)) AND (i_category_id#15 = category_id#53)), Statistics(sizeInBytes=7.0 MiB, rowCount=2.02E+5)
:- Project [i_item_sk#4, i_brand_id#11, i_class_id#13, i_category_id#15], Statistics(sizeInBytes=4.6 MiB, rowCount=2.02E+5)
: +- Filter ((isnotnull(i_brand_id#11) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#15)), Statistics(sizeInBytes=84.6 MiB, rowCount=2.02E+5)
: +- Relation spark_catalog.default.item[i_item_sk#4,i_item_id#5,i_rec_start_date#6,i_rec_end_date#7,i_item_desc#8,i_current_price#9,i_wholesale_cost#10,i_brand_id#11,i_brand#12,i_class_id#13,i_class#14,i_category_id#15,i_category#16,i_manufact_id#17,i_manufact#18,i_size#19,i_formulation#20,i_color#21,i_units#22,i_container#23,i_manager_id#24,i_product_name#25] parquet, Statistics(sizeInBytes=85.2 MiB, rowCount=2.04E+5)
+- Aggregate [brand_id#51, class_id#52, category_id#53], [brand_id#51, class_id#52, category_id#53], Statistics(sizeInBytes=2.6 MiB, rowCount=1.37E+5)
+- Project [i_brand_id#62 AS brand_id#51, i_class_id#64 AS class_id#52, i_category_id#66 AS category_id#53], Statistics(sizeInBytes=3.9 MiB, rowCount=2.02E+5)
+- Filter ((isnotnull(i_brand_id#62) AND isnotnull(i_class_id#64)) AND isnotnull(i_category_id#66)), Statistics(sizeInBytes=84.6 MiB, rowCount=2.02E+5)
+- Relation spark_catalog.default.item[i_item_sk#55,i_item_id#56,i_rec_start_date#57,i_rec_end_date#58,i_item_desc#59,i_current_price#60,i_wholesale_cost#61,i_brand_id#62,i_brand#63,i_class_id#64,i_class#65,i_category_id#66,i_category#67,i_manufact_id#68,i_manufact#69,i_size#70,i_formulation#71,i_color#72,i_units#73,i_container#74,i_manager_id#75,i_product_name#76] parquet, Statistics(sizeInBytes=85.2 MiB, rowCount=2.04E+5)
```
### Why are the changes needed?
Plan more broadcast joins to improve query performance.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Unit test and TPC-DS benchmark test.
SQL Before this PR(Seconds) After this PR(Seconds)
q14a 187 164
Closes #39923 from wankunde/SPARK-39851.
Lead-authored-by: wankunde <wa...@163.com>
Co-authored-by: Kun Wan <wa...@apache.org>
Signed-off-by: Dongjoon Hyun <do...@apache.org>
---
.../logical/statsEstimation/JoinEstimation.scala | 22 +-
.../SizeInBytesOnlyStatsPlanVisitor.scala | 15 +-
.../BasicStatsEstimationSuite.scala | 33 +-
.../approved-plans-v1_4/q14a.sf100/explain.txt | 678 ++++++++----------
.../approved-plans-v1_4/q14a.sf100/simplified.txt | 328 ++++-----
.../approved-plans-v1_4/q14b.sf100/explain.txt | 631 ++++++++--------
.../approved-plans-v1_4/q14b.sf100/simplified.txt | 335 ++++-----
.../approved-plans-v2_7/q14.sf100/explain.txt | 631 ++++++++--------
.../approved-plans-v2_7/q14.sf100/simplified.txt | 335 ++++-----
.../approved-plans-v2_7/q14a.sf100/explain.txt | 790 ++++++++++-----------
.../approved-plans-v2_7/q14a.sf100/simplified.txt | 358 +++++-----
11 files changed, 1936 insertions(+), 2220 deletions(-)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala
index 30f10970d1a..c6e76df1b31 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala
@@ -21,7 +21,7 @@ import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import org.apache.spark.internal.Logging
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Expression}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Expression, ExpressionSet}
import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Histogram, Join, Statistics}
@@ -56,10 +56,13 @@ case class JoinEstimation(join: Join) extends Logging {
case _ if !rowCountsExist(join.left, join.right) =>
None
- case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, _, _, _) =>
+ case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, _) =>
// 1. Compute join selectivity
val joinKeyPairs = extractJoinKeysWithColStats(leftKeys, rightKeys)
- val (numInnerJoinedRows, keyStatsAfterJoin) = computeCardinalityAndStats(joinKeyPairs)
+ val leftSideUniqueness = left.distinctKeys.exists(_.subsetOf(ExpressionSet(leftKeys)))
+ val rightSideUniqueness = right.distinctKeys.exists(_.subsetOf(ExpressionSet(rightKeys)))
+ val (numInnerJoinedRows, keyStatsAfterJoin) =
+ computeCardinalityAndStats(joinKeyPairs, leftSideUniqueness, rightSideUniqueness)
// 2. Estimate the number of output rows
val leftRows = leftStats.rowCount.get
@@ -175,10 +178,17 @@ case class JoinEstimation(join: Join) extends Logging {
* @return join cardinality, and column stats for join keys after the join
*/
// scalastyle:on
- private def computeCardinalityAndStats(keyPairs: Seq[(AttributeReference, AttributeReference)])
- : (BigInt, AttributeMap[ColumnStat]) = {
+ private def computeCardinalityAndStats(
+ keyPairs: Seq[(AttributeReference, AttributeReference)],
+ leftSideUniqueness: Boolean,
+ rightSideUniqueness: Boolean): (BigInt, AttributeMap[ColumnStat]) = {
// If there's no column stats available for join keys, estimate as cartesian product.
- var joinCard: BigInt = leftStats.rowCount.get * rightStats.rowCount.get
+ var joinCard: BigInt = (leftSideUniqueness, rightSideUniqueness) match {
+ case (true, true) => leftStats.rowCount.get.min(rightStats.rowCount.get)
+ case (true, false) => rightStats.rowCount.get
+ case (false, true) => leftStats.rowCount.get
+ case _ => leftStats.rowCount.get * rightStats.rowCount.get
+ }
val keyStatsAfterJoin = new mutable.HashMap[Attribute, ColumnStat]()
var i = 0
while(i < keyPairs.length && joinCard != 0) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala
index 77c728ba7c5..14cce02d99b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala
@@ -17,8 +17,9 @@
package org.apache.spark.sql.catalyst.plans.logical.statsEstimation
-import org.apache.spark.sql.catalyst.expressions.AttributeMap
-import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi}
+import org.apache.spark.sql.catalyst.expressions.{AttributeMap, ExpressionSet}
+import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
+import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
/**
@@ -112,6 +113,16 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] {
case LeftAnti | LeftSemi =>
// LeftSemi and LeftAnti won't ever be bigger than left
p.left.stats
+ case Inner | LeftOuter | RightOuter | FullOuter =>
+ p match {
+ case ExtractEquiJoinKeys(_, leftKeys, rightKeys, _, _, left, right, _)
+ if left.distinctKeys.exists(_.subsetOf(ExpressionSet(leftKeys))) ||
+ right.distinctKeys.exists(_.subsetOf(ExpressionSet(rightKeys))) =>
+ // The sizeInBytes should be > 1 because sizeInBytes * 1 != sizeInBytes + 1.
+ Statistics(sizeInBytes = p.children.map(_.stats.sizeInBytes).filter(_ > 1L).sum)
+ case _ =>
+ default(p)
+ }
case _ =>
default(p)
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
index 4362e0c5172..33e521eb65a 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.analysis.ResolvedNamespace
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeMap, AttributeReference, Literal, SortOrder}
-import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.connector.catalog.SupportsNamespaces
import org.apache.spark.sql.internal.SQLConf
@@ -328,6 +328,37 @@ class BasicStatsEstimationSuite extends PlanTest with StatsEstimationTestBase {
expectedStatsCboOff = Statistics(sizeInBytes = sizeInBytes))
}
+ test("SPARK-39851: Improve join stats estimation if one side can keep uniqueness") {
+ val brandId = attr("brand_id")
+ val classId = attr("class_id")
+ val aliasedBrandId = brandId.as("new_brand_id")
+ val aliasedClassId = classId.as("new_class_id")
+
+ val tableSize = 4059900
+ val tableRowCnt = 202995
+
+ val tbl = StatsTestPlan(
+ outputList = Seq(brandId, classId),
+ size = Some(tableSize),
+ rowCount = tableRowCnt,
+ attributeStats =
+ AttributeMap(Seq(
+ brandId -> ColumnStat(Some(858), Some(101001), Some(1016017), Some(0), Some(4), Some(4)),
+ classId -> ColumnStat(Some(16), Some(1), Some(16), Some(0), Some(4), Some(4)))))
+
+ val join = Join(
+ tbl,
+ tbl.groupBy(brandId, classId)(aliasedBrandId, aliasedClassId),
+ Inner,
+ Some(brandId === aliasedBrandId.toAttribute && classId === aliasedClassId.toAttribute),
+ JoinHint.NONE)
+
+ checkStats(
+ join,
+ expectedStatsCboOn = Statistics(4871880, Some(tableRowCnt), join.stats.attributeStats),
+ expectedStatsCboOff = Statistics(sizeInBytes = 4059900 * 2))
+ }
+
test("row size and column stats estimation for sort") {
val columnInfo = AttributeMap(
Seq(
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt
index 8b962208051..e350459c60c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt
@@ -1,127 +1,115 @@
== Physical Plan ==
-TakeOrderedAndProject (123)
-+- * HashAggregate (122)
- +- Exchange (121)
- +- * HashAggregate (120)
- +- * Expand (119)
- +- Union (118)
- :- * Project (79)
- : +- * Filter (78)
- : +- * HashAggregate (77)
- : +- Exchange (76)
- : +- * HashAggregate (75)
- : +- * Project (74)
- : +- * BroadcastHashJoin Inner BuildRight (73)
- : :- * Project (63)
- : : +- * BroadcastHashJoin Inner BuildRight (62)
- : : :- * SortMergeJoin LeftSemi (60)
- : : : :- * Sort (5)
- : : : : +- Exchange (4)
- : : : : +- * Filter (3)
- : : : : +- * ColumnarToRow (2)
- : : : : +- Scan parquet spark_catalog.default.store_sales (1)
- : : : +- * Sort (59)
- : : : +- Exchange (58)
- : : : +- * Project (57)
- : : : +- * BroadcastHashJoin Inner BuildRight (56)
- : : : :- * Filter (8)
- : : : : +- * ColumnarToRow (7)
- : : : : +- Scan parquet spark_catalog.default.item (6)
- : : : +- BroadcastExchange (55)
- : : : +- * SortMergeJoin LeftSemi (54)
- : : : :- * Sort (42)
- : : : : +- Exchange (41)
- : : : : +- * HashAggregate (40)
- : : : : +- Exchange (39)
- : : : : +- * HashAggregate (38)
- : : : : +- * Project (37)
- : : : : +- * BroadcastHashJoin Inner BuildRight (36)
- : : : : :- * Project (14)
- : : : : : +- * BroadcastHashJoin Inner BuildRight (13)
- : : : : : :- * Filter (11)
- : : : : : : +- * ColumnarToRow (10)
- : : : : : : +- Scan parquet spark_catalog.default.store_sales (9)
- : : : : : +- ReusedExchange (12)
- : : : : +- BroadcastExchange (35)
- : : : : +- * SortMergeJoin LeftSemi (34)
- : : : : :- * Sort (19)
- : : : : : +- Exchange (18)
- : : : : : +- * Filter (17)
- : : : : : +- * ColumnarToRow (16)
- : : : : : +- Scan parquet spark_catalog.default.item (15)
- : : : : +- * Sort (33)
- : : : : +- Exchange (32)
- : : : : +- * Project (31)
- : : : : +- * BroadcastHashJoin Inner BuildRight (30)
- : : : : :- * Project (25)
- : : : : : +- * BroadcastHashJoin Inner BuildRight (24)
- : : : : : :- * Filter (22)
- : : : : : : +- * ColumnarToRow (21)
- : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (20)
- : : : : : +- ReusedExchange (23)
- : : : : +- BroadcastExchange (29)
- : : : : +- * Filter (28)
- : : : : +- * ColumnarToRow (27)
- : : : : +- Scan parquet spark_catalog.default.item (26)
- : : : +- * Sort (53)
- : : : +- Exchange (52)
- : : : +- * Project (51)
- : : : +- * BroadcastHashJoin Inner BuildRight (50)
- : : : :- * Project (48)
- : : : : +- * BroadcastHashJoin Inner BuildRight (47)
- : : : : :- * Filter (45)
- : : : : : +- * ColumnarToRow (44)
- : : : : : +- Scan parquet spark_catalog.default.web_sales (43)
- : : : : +- ReusedExchange (46)
- : : : +- ReusedExchange (49)
- : : +- ReusedExchange (61)
- : +- BroadcastExchange (72)
- : +- * SortMergeJoin LeftSemi (71)
- : :- * Sort (68)
- : : +- Exchange (67)
- : : +- * Filter (66)
- : : +- * ColumnarToRow (65)
- : : +- Scan parquet spark_catalog.default.item (64)
- : +- * Sort (70)
- : +- ReusedExchange (69)
- :- * Project (98)
- : +- * Filter (97)
- : +- * HashAggregate (96)
- : +- Exchange (95)
- : +- * HashAggregate (94)
- : +- * Project (93)
- : +- * BroadcastHashJoin Inner BuildRight (92)
- : :- * Project (90)
- : : +- * BroadcastHashJoin Inner BuildRight (89)
- : : :- * SortMergeJoin LeftSemi (87)
- : : : :- * Sort (84)
- : : : : +- Exchange (83)
- : : : : +- * Filter (82)
- : : : : +- * ColumnarToRow (81)
- : : : : +- Scan parquet spark_catalog.default.catalog_sales (80)
- : : : +- * Sort (86)
- : : : +- ReusedExchange (85)
- : : +- ReusedExchange (88)
- : +- ReusedExchange (91)
- +- * Project (117)
- +- * Filter (116)
- +- * HashAggregate (115)
- +- Exchange (114)
- +- * HashAggregate (113)
- +- * Project (112)
- +- * BroadcastHashJoin Inner BuildRight (111)
- :- * Project (109)
- : +- * BroadcastHashJoin Inner BuildRight (108)
- : :- * SortMergeJoin LeftSemi (106)
- : : :- * Sort (103)
- : : : +- Exchange (102)
- : : : +- * Filter (101)
- : : : +- * ColumnarToRow (100)
- : : : +- Scan parquet spark_catalog.default.web_sales (99)
- : : +- * Sort (105)
- : : +- ReusedExchange (104)
- : +- ReusedExchange (107)
- +- ReusedExchange (110)
+TakeOrderedAndProject (111)
++- * HashAggregate (110)
+ +- Exchange (109)
+ +- * HashAggregate (108)
+ +- * Expand (107)
+ +- Union (106)
+ :- * Project (73)
+ : +- * 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)
+ :- * Project (89)
+ : +- * Filter (88)
+ : +- * HashAggregate (87)
+ : +- Exchange (86)
+ : +- * HashAggregate (85)
+ : +- * Project (84)
+ : +- * BroadcastHashJoin Inner BuildRight (83)
+ : :- * Project (81)
+ : : +- * BroadcastHashJoin Inner BuildRight (80)
+ : : :- * BroadcastHashJoin LeftSemi BuildRight (78)
+ : : : :- * Filter (76)
+ : : : : +- * ColumnarToRow (75)
+ : : : : +- Scan parquet spark_catalog.default.catalog_sales (74)
+ : : : +- ReusedExchange (77)
+ : : +- ReusedExchange (79)
+ : +- ReusedExchange (82)
+ +- * Project (105)
+ +- * Filter (104)
+ +- * HashAggregate (103)
+ +- Exchange (102)
+ +- * HashAggregate (101)
+ +- * Project (100)
+ +- * BroadcastHashJoin Inner BuildRight (99)
+ :- * Project (97)
+ : +- * BroadcastHashJoin Inner BuildRight (96)
+ : :- * BroadcastHashJoin LeftSemi BuildRight (94)
+ : : :- * Filter (92)
+ : : : +- * ColumnarToRow (91)
+ : : : +- Scan parquet spark_catalog.default.web_sales (90)
+ : : +- ReusedExchange (93)
+ : +- ReusedExchange (95)
+ +- ReusedExchange (98)
(1) Scan parquet spark_catalog.default.store_sales
@@ -132,36 +120,28 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol
PushedFilters: [IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)>
-(2) ColumnarToRow [codegen id : 1]
+(2) ColumnarToRow [codegen id : 37]
Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-(3) Filter [codegen id : 1]
+(3) Filter [codegen id : 37]
Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
Condition : isnotnull(ss_item_sk#1)
-(4) Exchange
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1]
-
-(5) Sort [codegen id : 2]
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0
-
-(6) Scan parquet spark_catalog.default.item
+(4) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(7) ColumnarToRow [codegen id : 19]
+(5) ColumnarToRow [codegen id : 17]
Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
-(8) Filter [codegen id : 19]
+(6) Filter [codegen id : 17]
Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9))
-(9) Scan parquet spark_catalog.default.store_sales
+(7) Scan parquet spark_catalog.default.store_sales
Output [2]: [ss_item_sk#10, ss_sold_date_sk#11]
Batched: true
Location: InMemoryFileIndex []
@@ -169,49 +149,49 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so
PushedFilters: [IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int>
-(10) ColumnarToRow [codegen id : 11]
+(8) ColumnarToRow [codegen id : 9]
Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
-(11) Filter [codegen id : 11]
+(9) Filter [codegen id : 9]
Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
Condition : isnotnull(ss_item_sk#10)
-(12) ReusedExchange [Reuses operator id: 152]
+(10) ReusedExchange [Reuses operator id: 140]
Output [1]: [d_date_sk#13]
-(13) BroadcastHashJoin [codegen id : 11]
+(11) BroadcastHashJoin [codegen id : 9]
Left keys [1]: [ss_sold_date_sk#11]
Right keys [1]: [d_date_sk#13]
Join type: Inner
Join condition: None
-(14) Project [codegen id : 11]
+(12) Project [codegen id : 9]
Output [1]: [ss_item_sk#10]
Input [3]: [ss_item_sk#10, ss_sold_date_sk#11, d_date_sk#13]
-(15) Scan parquet spark_catalog.default.item
+(13) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(16) ColumnarToRow [codegen id : 4]
+(14) ColumnarToRow [codegen id : 2]
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-(17) Filter [codegen id : 4]
+(15) Filter [codegen id : 2]
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
Condition : (((isnotnull(i_item_sk#14) AND isnotnull(i_brand_id#15)) AND isnotnull(i_class_id#16)) AND isnotnull(i_category_id#17))
-(18) Exchange
+(16) Exchange
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=2]
+Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=1]
-(19) Sort [codegen id : 5]
+(17) Sort [codegen id : 3]
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
Arguments: [coalesce(i_brand_id#15, 0) ASC NULLS FIRST, isnull(i_brand_id#15) ASC NULLS FIRST, coalesce(i_class_id#16, 0) ASC NULLS FIRST, isnull(i_class_id#16) ASC NULLS FIRST, coalesce(i_category_id#17, 0) ASC NULLS FIRST, isnull(i_category_id#17) ASC NULLS FIRST], false, 0
-(20) Scan parquet spark_catalog.default.catalog_sales
+(18) Scan parquet spark_catalog.default.catalog_sales
Output [2]: [cs_item_sk#18, cs_sold_date_sk#19]
Batched: true
Location: InMemoryFileIndex []
@@ -219,109 +199,109 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_so
PushedFilters: [IsNotNull(cs_item_sk)]
ReadSchema: struct<cs_item_sk:int>
-(21) ColumnarToRow [codegen id : 8]
+(19) ColumnarToRow [codegen id : 6]
Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
-(22) Filter [codegen id : 8]
+(20) Filter [codegen id : 6]
Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
Condition : isnotnull(cs_item_sk#18)
-(23) ReusedExchange [Reuses operator id: 152]
+(21) ReusedExchange [Reuses operator id: 140]
Output [1]: [d_date_sk#20]
-(24) BroadcastHashJoin [codegen id : 8]
+(22) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [cs_sold_date_sk#19]
Right keys [1]: [d_date_sk#20]
Join type: Inner
Join condition: None
-(25) Project [codegen id : 8]
+(23) Project [codegen id : 6]
Output [1]: [cs_item_sk#18]
Input [3]: [cs_item_sk#18, cs_sold_date_sk#19, d_date_sk#20]
-(26) Scan parquet spark_catalog.default.item
+(24) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(27) ColumnarToRow [codegen id : 7]
+(25) ColumnarToRow [codegen id : 5]
Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-(28) Filter [codegen id : 7]
+(26) Filter [codegen id : 5]
Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
Condition : isnotnull(i_item_sk#21)
-(29) BroadcastExchange
+(27) BroadcastExchange
Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2]
-(30) BroadcastHashJoin [codegen id : 8]
+(28) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [cs_item_sk#18]
Right keys [1]: [i_item_sk#21]
Join type: Inner
Join condition: None
-(31) Project [codegen id : 8]
+(29) Project [codegen id : 6]
Output [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
Input [5]: [cs_item_sk#18, i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-(32) Exchange
+(30) Exchange
Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=4]
+Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=3]
-(33) Sort [codegen id : 9]
+(31) Sort [codegen id : 7]
Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
Arguments: [coalesce(i_brand_id#22, 0) ASC NULLS FIRST, isnull(i_brand_id#22) ASC NULLS FIRST, coalesce(i_class_id#23, 0) ASC NULLS FIRST, isnull(i_class_id#23) ASC NULLS FIRST, coalesce(i_category_id#24, 0) ASC NULLS FIRST, isnull(i_category_id#24) ASC NULLS FIRST], false, 0
-(34) SortMergeJoin [codegen id : 10]
+(32) SortMergeJoin [codegen id : 8]
Left keys [6]: [coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17)]
Right keys [6]: [coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24)]
Join type: LeftSemi
Join condition: None
-(35) BroadcastExchange
+(33) BroadcastExchange
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
-(36) BroadcastHashJoin [codegen id : 11]
+(34) BroadcastHashJoin [codegen id : 9]
Left keys [1]: [ss_item_sk#10]
Right keys [1]: [i_item_sk#14]
Join type: Inner
Join condition: None
-(37) Project [codegen id : 11]
+(35) Project [codegen id : 9]
Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27]
Input [5]: [ss_item_sk#10, i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-(38) HashAggregate [codegen id : 11]
+(36) HashAggregate [codegen id : 9]
Input [3]: [brand_id#25, class_id#26, category_id#27]
Keys [3]: [brand_id#25, class_id#26, category_id#27]
Functions: []
Aggregate Attributes: []
Results [3]: [brand_id#25, class_id#26, category_id#27]
-(39) Exchange
+(37) Exchange
Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=5]
-(40) HashAggregate [codegen id : 12]
+(38) HashAggregate [codegen id : 10]
Input [3]: [brand_id#25, class_id#26, category_id#27]
Keys [3]: [brand_id#25, class_id#26, category_id#27]
Functions: []
Aggregate Attributes: []
Results [3]: [brand_id#25, class_id#26, category_id#27]
-(41) Exchange
+(39) Exchange
Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=7]
+Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=6]
-(42) Sort [codegen id : 13]
+(40) Sort [codegen id : 11]
Input [3]: [brand_id#25, class_id#26, category_id#27]
Arguments: [coalesce(brand_id#25, 0) ASC NULLS FIRST, isnull(brand_id#25) ASC NULLS FIRST, coalesce(class_id#26, 0) ASC NULLS FIRST, isnull(class_id#26) ASC NULLS FIRST, coalesce(category_id#27, 0) ASC NULLS FIRST, isnull(category_id#27) ASC NULLS FIRST], false, 0
-(43) Scan parquet spark_catalog.default.web_sales
+(41) Scan parquet spark_catalog.default.web_sales
Output [2]: [ws_item_sk#28, ws_sold_date_sk#29]
Batched: true
Location: InMemoryFileIndex []
@@ -329,170 +309,154 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so
PushedFilters: [IsNotNull(ws_item_sk)]
ReadSchema: struct<ws_item_sk:int>
-(44) ColumnarToRow [codegen id : 16]
+(42) ColumnarToRow [codegen id : 14]
Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
-(45) Filter [codegen id : 16]
+(43) Filter [codegen id : 14]
Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
Condition : isnotnull(ws_item_sk#28)
-(46) ReusedExchange [Reuses operator id: 152]
+(44) ReusedExchange [Reuses operator id: 140]
Output [1]: [d_date_sk#30]
-(47) BroadcastHashJoin [codegen id : 16]
+(45) BroadcastHashJoin [codegen id : 14]
Left keys [1]: [ws_sold_date_sk#29]
Right keys [1]: [d_date_sk#30]
Join type: Inner
Join condition: None
-(48) Project [codegen id : 16]
+(46) Project [codegen id : 14]
Output [1]: [ws_item_sk#28]
Input [3]: [ws_item_sk#28, ws_sold_date_sk#29, d_date_sk#30]
-(49) ReusedExchange [Reuses operator id: 29]
+(47) ReusedExchange [Reuses operator id: 27]
Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
-(50) BroadcastHashJoin [codegen id : 16]
+(48) BroadcastHashJoin [codegen id : 14]
Left keys [1]: [ws_item_sk#28]
Right keys [1]: [i_item_sk#31]
Join type: Inner
Join condition: None
-(51) Project [codegen id : 16]
+(49) Project [codegen id : 14]
Output [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
Input [5]: [ws_item_sk#28, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
-(52) Exchange
+(50) Exchange
Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
-Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=8]
+Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=7]
-(53) Sort [codegen id : 17]
+(51) Sort [codegen id : 15]
Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
Arguments: [coalesce(i_brand_id#32, 0) ASC NULLS FIRST, isnull(i_brand_id#32) ASC NULLS FIRST, coalesce(i_class_id#33, 0) ASC NULLS FIRST, isnull(i_class_id#33) ASC NULLS FIRST, coalesce(i_category_id#34, 0) ASC NULLS FIRST, isnull(i_category_id#34) ASC NULLS FIRST], false, 0
-(54) SortMergeJoin [codegen id : 18]
+(52) SortMergeJoin [codegen id : 16]
Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)]
Right keys [6]: [coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34)]
Join type: LeftSemi
Join condition: None
-(55) BroadcastExchange
+(53) BroadcastExchange
Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=9]
+Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=8]
-(56) BroadcastHashJoin [codegen id : 19]
+(54) BroadcastHashJoin [codegen id : 17]
Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Right keys [3]: [brand_id#25, class_id#26, category_id#27]
Join type: Inner
Join condition: None
-(57) Project [codegen id : 19]
+(55) Project [codegen id : 17]
Output [1]: [i_item_sk#6 AS ss_item_sk#35]
Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27]
-(58) Exchange
-Input [1]: [ss_item_sk#35]
-Arguments: hashpartitioning(ss_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=10]
-
-(59) Sort [codegen id : 20]
+(56) BroadcastExchange
Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9]
-(60) SortMergeJoin [codegen id : 43]
+(57) BroadcastHashJoin [codegen id : 37]
Left keys [1]: [ss_item_sk#1]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(61) ReusedExchange [Reuses operator id: 147]
+(58) ReusedExchange [Reuses operator id: 135]
Output [1]: [d_date_sk#36]
-(62) BroadcastHashJoin [codegen id : 43]
+(59) BroadcastHashJoin [codegen id : 37]
Left keys [1]: [ss_sold_date_sk#4]
Right keys [1]: [d_date_sk#36]
Join type: Inner
Join condition: None
-(63) Project [codegen id : 43]
+(60) Project [codegen id : 37]
Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3]
Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36]
-(64) Scan parquet spark_catalog.default.item
+(61) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(65) ColumnarToRow [codegen id : 22]
+(62) ColumnarToRow [codegen id : 36]
Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-(66) Filter [codegen id : 22]
+(63) Filter [codegen id : 36]
Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
Condition : isnotnull(i_item_sk#37)
-(67) Exchange
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: hashpartitioning(i_item_sk#37, 5), ENSURE_REQUIREMENTS, [plan_id=11]
-
-(68) Sort [codegen id : 23]
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: [i_item_sk#37 ASC NULLS FIRST], false, 0
-
-(69) ReusedExchange [Reuses operator id: 58]
+(64) ReusedExchange [Reuses operator id: 56]
Output [1]: [ss_item_sk#35]
-(70) Sort [codegen id : 41]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(71) SortMergeJoin [codegen id : 42]
+(65) BroadcastHashJoin [codegen id : 36]
Left keys [1]: [i_item_sk#37]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(72) BroadcastExchange
+(66) BroadcastExchange
Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10]
-(73) BroadcastHashJoin [codegen id : 43]
+(67) BroadcastHashJoin [codegen id : 37]
Left keys [1]: [ss_item_sk#1]
Right keys [1]: [i_item_sk#37]
Join type: Inner
Join condition: None
-(74) Project [codegen id : 43]
+(68) Project [codegen id : 37]
Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-(75) HashAggregate [codegen id : 43]
+(69) HashAggregate [codegen id : 37]
Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
-(76) Exchange
+(70) Exchange
Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
-Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=13]
+Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11]
-(77) HashAggregate [codegen id : 44]
+(71) HashAggregate [codegen id : 38]
Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
Results [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#49, count(1)#48 AS number_sales#50]
-(78) Filter [codegen id : 44]
+(72) Filter [codegen id : 38]
Input [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sales#49, number_sales#50]
Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6))))
-(79) Project [codegen id : 44]
+(73) Project [codegen id : 38]
Output [6]: [sales#49, number_sales#50, store AS channel#53, i_brand_id#38 AS i_brand_id#54, i_class_id#39 AS i_class_id#55, i_category_id#40 AS i_category_id#56]
Input [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sales#49, number_sales#50]
-(80) Scan parquet spark_catalog.default.catalog_sales
+(74) Scan parquet spark_catalog.default.catalog_sales
Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60]
Batched: true
Location: InMemoryFileIndex []
@@ -500,87 +464,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so
PushedFilters: [IsNotNull(cs_item_sk)]
ReadSchema: struct<cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2)>
-(81) ColumnarToRow [codegen id : 45]
+(75) ColumnarToRow [codegen id : 75]
Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60]
-(82) Filter [codegen id : 45]
+(76) Filter [codegen id : 75]
Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60]
Condition : isnotnull(cs_item_sk#57)
-(83) Exchange
-Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60]
-Arguments: hashpartitioning(cs_item_sk#57, 5), ENSURE_REQUIREMENTS, [plan_id=14]
-
-(84) Sort [codegen id : 46]
-Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60]
-Arguments: [cs_item_sk#57 ASC NULLS FIRST], false, 0
-
-(85) ReusedExchange [Reuses operator id: 58]
+(77) ReusedExchange [Reuses operator id: 56]
Output [1]: [ss_item_sk#35]
-(86) Sort [codegen id : 64]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(87) SortMergeJoin [codegen id : 87]
+(78) BroadcastHashJoin [codegen id : 75]
Left keys [1]: [cs_item_sk#57]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(88) ReusedExchange [Reuses operator id: 147]
+(79) ReusedExchange [Reuses operator id: 135]
Output [1]: [d_date_sk#61]
-(89) BroadcastHashJoin [codegen id : 87]
+(80) BroadcastHashJoin [codegen id : 75]
Left keys [1]: [cs_sold_date_sk#60]
Right keys [1]: [d_date_sk#61]
Join type: Inner
Join condition: None
-(90) Project [codegen id : 87]
+(81) Project [codegen id : 75]
Output [3]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59]
Input [5]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, d_date_sk#61]
-(91) ReusedExchange [Reuses operator id: 72]
+(82) ReusedExchange [Reuses operator id: 66]
Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65]
-(92) BroadcastHashJoin [codegen id : 87]
+(83) BroadcastHashJoin [codegen id : 75]
Left keys [1]: [cs_item_sk#57]
Right keys [1]: [i_item_sk#62]
Join type: Inner
Join condition: None
-(93) Project [codegen id : 87]
+(84) Project [codegen id : 75]
Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65]
Input [7]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65]
-(94) HashAggregate [codegen id : 87]
+(85) HashAggregate [codegen id : 75]
Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65]
Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65]
Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)]
Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68]
Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#69, isEmpty#70, count#71]
-(95) Exchange
+(86) Exchange
Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#69, isEmpty#70, count#71]
-Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=15]
+Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=12]
-(96) HashAggregate [codegen id : 88]
+(87) HashAggregate [codegen id : 76]
Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#69, isEmpty#70, count#71]
Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65]
Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)]
Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72, count(1)#73]
Results [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72 AS sales#74, count(1)#73 AS number_sales#75]
-(97) Filter [codegen id : 88]
+(88) Filter [codegen id : 76]
Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#74, number_sales#75]
Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6))))
-(98) Project [codegen id : 88]
+(89) Project [codegen id : 76]
Output [6]: [sales#74, number_sales#75, catalog AS channel#76, i_brand_id#63, i_class_id#64, i_category_id#65]
Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#74, number_sales#75]
-(99) Scan parquet spark_catalog.default.web_sales
+(90) Scan parquet spark_catalog.default.web_sales
Output [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80]
Batched: true
Location: InMemoryFileIndex []
@@ -588,303 +540,291 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_so
PushedFilters: [IsNotNull(ws_item_sk)]
ReadSchema: struct<ws_item_sk:int,ws_quantity:int,ws_list_price:decimal(7,2)>
-(100) ColumnarToRow [codegen id : 89]
+(91) ColumnarToRow [codegen id : 113]
Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80]
-(101) Filter [codegen id : 89]
+(92) Filter [codegen id : 113]
Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80]
Condition : isnotnull(ws_item_sk#77)
-(102) Exchange
-Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80]
-Arguments: hashpartitioning(ws_item_sk#77, 5), ENSURE_REQUIREMENTS, [plan_id=16]
-
-(103) Sort [codegen id : 90]
-Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80]
-Arguments: [ws_item_sk#77 ASC NULLS FIRST], false, 0
-
-(104) ReusedExchange [Reuses operator id: 58]
+(93) ReusedExchange [Reuses operator id: 56]
Output [1]: [ss_item_sk#35]
-(105) Sort [codegen id : 108]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(106) SortMergeJoin [codegen id : 131]
+(94) BroadcastHashJoin [codegen id : 113]
Left keys [1]: [ws_item_sk#77]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(107) ReusedExchange [Reuses operator id: 147]
+(95) ReusedExchange [Reuses operator id: 135]
Output [1]: [d_date_sk#81]
-(108) BroadcastHashJoin [codegen id : 131]
+(96) BroadcastHashJoin [codegen id : 113]
Left keys [1]: [ws_sold_date_sk#80]
Right keys [1]: [d_date_sk#81]
Join type: Inner
Join condition: None
-(109) Project [codegen id : 131]
+(97) Project [codegen id : 113]
Output [3]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79]
Input [5]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, d_date_sk#81]
-(110) ReusedExchange [Reuses operator id: 72]
+(98) ReusedExchange [Reuses operator id: 66]
Output [4]: [i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85]
-(111) BroadcastHashJoin [codegen id : 131]
+(99) BroadcastHashJoin [codegen id : 113]
Left keys [1]: [ws_item_sk#77]
Right keys [1]: [i_item_sk#82]
Join type: Inner
Join condition: None
-(112) Project [codegen id : 131]
+(100) Project [codegen id : 113]
Output [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85]
Input [7]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85]
-(113) HashAggregate [codegen id : 131]
+(101) HashAggregate [codegen id : 113]
Input [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85]
Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85]
Functions [2]: [partial_sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), partial_count(1)]
Aggregate Attributes [3]: [sum#86, isEmpty#87, count#88]
Results [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#89, isEmpty#90, count#91]
-(114) Exchange
+(102) Exchange
Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#89, isEmpty#90, count#91]
-Arguments: hashpartitioning(i_brand_id#83, i_class_id#84, i_category_id#85, 5), ENSURE_REQUIREMENTS, [plan_id=17]
+Arguments: hashpartitioning(i_brand_id#83, i_class_id#84, i_category_id#85, 5), ENSURE_REQUIREMENTS, [plan_id=13]
-(115) HashAggregate [codegen id : 132]
+(103) HashAggregate [codegen id : 114]
Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#89, isEmpty#90, count#91]
Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85]
Functions [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), count(1)]
Aggregate Attributes [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92, count(1)#93]
Results [5]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92 AS sales#94, count(1)#93 AS number_sales#95]
-(116) Filter [codegen id : 132]
+(104) Filter [codegen id : 114]
Input [5]: [i_brand_id#83, i_class_id#84, i_category_id#85, sales#94, number_sales#95]
Condition : (isnotnull(sales#94) AND (cast(sales#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6))))
-(117) Project [codegen id : 132]
+(105) Project [codegen id : 114]
Output [6]: [sales#94, number_sales#95, web AS channel#96, i_brand_id#83, i_class_id#84, i_category_id#85]
Input [5]: [i_brand_id#83, i_class_id#84, i_category_id#85, sales#94, number_sales#95]
-(118) Union
+(106) Union
-(119) Expand [codegen id : 133]
+(107) Expand [codegen id : 115]
Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56]
Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101]
-(120) HashAggregate [codegen id : 133]
+(108) HashAggregate [codegen id : 115]
Input [7]: [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101]
Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101]
Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)]
Aggregate Attributes [3]: [sum#102, isEmpty#103, sum#104]
Results [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107]
-(121) Exchange
+(109) Exchange
Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107]
-Arguments: hashpartitioning(channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=18]
+Arguments: hashpartitioning(channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=14]
-(122) HashAggregate [codegen id : 134]
+(110) HashAggregate [codegen id : 116]
Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107]
Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101]
Functions [2]: [sum(sales#49), sum(number_sales#50)]
Aggregate Attributes [2]: [sum(sales#49)#108, sum(number_sales#50)#109]
Results [6]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales#49)#108 AS sum(sales)#110, sum(number_sales#50)#109 AS sum(number_sales)#111]
-(123) TakeOrderedAndProject
+(111) TakeOrderedAndProject
Input [6]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales)#110, sum(number_sales)#111]
Arguments: 100, [channel#97 ASC NULLS FIRST, i_brand_id#98 ASC NULLS FIRST, i_class_id#99 ASC NULLS FIRST, i_category_id#100 ASC NULLS FIRST], [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales)#110, sum(number_sales)#111]
===== Subqueries =====
-Subquery:1 Hosting operator id = 78 Hosting Expression = Subquery scalar-subquery#51, [id=#52]
-* HashAggregate (142)
-+- Exchange (141)
- +- * HashAggregate (140)
- +- Union (139)
- :- * Project (128)
- : +- * BroadcastHashJoin Inner BuildRight (127)
- : :- * ColumnarToRow (125)
- : : +- Scan parquet spark_catalog.default.store_sales (124)
- : +- ReusedExchange (126)
- :- * Project (133)
- : +- * BroadcastHashJoin Inner BuildRight (132)
- : :- * ColumnarToRow (130)
- : : +- Scan parquet spark_catalog.default.catalog_sales (129)
- : +- ReusedExchange (131)
- +- * Project (138)
- +- * BroadcastHashJoin Inner BuildRight (137)
- :- * ColumnarToRow (135)
- : +- Scan parquet spark_catalog.default.web_sales (134)
- +- ReusedExchange (136)
-
-
-(124) Scan parquet spark_catalog.default.store_sales
+Subquery:1 Hosting operator id = 72 Hosting Expression = Subquery scalar-subquery#51, [id=#52]
+* HashAggregate (130)
++- Exchange (129)
+ +- * HashAggregate (128)
+ +- Union (127)
+ :- * Project (116)
+ : +- * BroadcastHashJoin Inner BuildRight (115)
+ : :- * ColumnarToRow (113)
+ : : +- Scan parquet spark_catalog.default.store_sales (112)
+ : +- ReusedExchange (114)
+ :- * Project (121)
+ : +- * BroadcastHashJoin Inner BuildRight (120)
+ : :- * ColumnarToRow (118)
+ : : +- Scan parquet spark_catalog.default.catalog_sales (117)
+ : +- ReusedExchange (119)
+ +- * Project (126)
+ +- * BroadcastHashJoin Inner BuildRight (125)
+ :- * ColumnarToRow (123)
+ : +- Scan parquet spark_catalog.default.web_sales (122)
+ +- ReusedExchange (124)
+
+
+(112) Scan parquet spark_catalog.default.store_sales
Output [3]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ss_sold_date_sk#114), dynamicpruningexpression(ss_sold_date_sk#114 IN dynamicpruning#12)]
ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)>
-(125) ColumnarToRow [codegen id : 2]
+(113) ColumnarToRow [codegen id : 2]
Input [3]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114]
-(126) ReusedExchange [Reuses operator id: 152]
+(114) ReusedExchange [Reuses operator id: 140]
Output [1]: [d_date_sk#115]
-(127) BroadcastHashJoin [codegen id : 2]
+(115) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [ss_sold_date_sk#114]
Right keys [1]: [d_date_sk#115]
Join type: Inner
Join condition: None
-(128) Project [codegen id : 2]
+(116) Project [codegen id : 2]
Output [2]: [ss_quantity#112 AS quantity#116, ss_list_price#113 AS list_price#117]
Input [4]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114, d_date_sk#115]
-(129) Scan parquet spark_catalog.default.catalog_sales
+(117) Scan parquet spark_catalog.default.catalog_sales
Output [3]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(cs_sold_date_sk#120), dynamicpruningexpression(cs_sold_date_sk#120 IN dynamicpruning#12)]
ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)>
-(130) ColumnarToRow [codegen id : 4]
+(118) ColumnarToRow [codegen id : 4]
Input [3]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120]
-(131) ReusedExchange [Reuses operator id: 152]
+(119) ReusedExchange [Reuses operator id: 140]
Output [1]: [d_date_sk#121]
-(132) BroadcastHashJoin [codegen id : 4]
+(120) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [cs_sold_date_sk#120]
Right keys [1]: [d_date_sk#121]
Join type: Inner
Join condition: None
-(133) Project [codegen id : 4]
+(121) Project [codegen id : 4]
Output [2]: [cs_quantity#118 AS quantity#122, cs_list_price#119 AS list_price#123]
Input [4]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120, d_date_sk#121]
-(134) Scan parquet spark_catalog.default.web_sales
+(122) Scan parquet spark_catalog.default.web_sales
Output [3]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ws_sold_date_sk#126), dynamicpruningexpression(ws_sold_date_sk#126 IN dynamicpruning#12)]
ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)>
-(135) ColumnarToRow [codegen id : 6]
+(123) ColumnarToRow [codegen id : 6]
Input [3]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126]
-(136) ReusedExchange [Reuses operator id: 152]
+(124) ReusedExchange [Reuses operator id: 140]
Output [1]: [d_date_sk#127]
-(137) BroadcastHashJoin [codegen id : 6]
+(125) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [ws_sold_date_sk#126]
Right keys [1]: [d_date_sk#127]
Join type: Inner
Join condition: None
-(138) Project [codegen id : 6]
+(126) Project [codegen id : 6]
Output [2]: [ws_quantity#124 AS quantity#128, ws_list_price#125 AS list_price#129]
Input [4]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126, d_date_sk#127]
-(139) Union
+(127) Union
-(140) HashAggregate [codegen id : 7]
+(128) HashAggregate [codegen id : 7]
Input [2]: [quantity#116, list_price#117]
Keys: []
Functions [1]: [partial_avg((cast(quantity#116 as decimal(10,0)) * list_price#117))]
Aggregate Attributes [2]: [sum#130, count#131]
Results [2]: [sum#132, count#133]
-(141) Exchange
+(129) Exchange
Input [2]: [sum#132, count#133]
-Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=19]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15]
-(142) HashAggregate [codegen id : 8]
+(130) HashAggregate [codegen id : 8]
Input [2]: [sum#132, count#133]
Keys: []
Functions [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))]
Aggregate Attributes [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))#134]
Results [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))#134 AS average_sales#135]
-Subquery:2 Hosting operator id = 124 Hosting Expression = ss_sold_date_sk#114 IN dynamicpruning#12
+Subquery:2 Hosting operator id = 112 Hosting Expression = ss_sold_date_sk#114 IN dynamicpruning#12
-Subquery:3 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#120 IN dynamicpruning#12
+Subquery:3 Hosting operator id = 117 Hosting Expression = cs_sold_date_sk#120 IN dynamicpruning#12
-Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#126 IN dynamicpruning#12
+Subquery:4 Hosting operator id = 122 Hosting Expression = ws_sold_date_sk#126 IN dynamicpruning#12
Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (147)
-+- * Project (146)
- +- * Filter (145)
- +- * ColumnarToRow (144)
- +- Scan parquet spark_catalog.default.date_dim (143)
+BroadcastExchange (135)
++- * Project (134)
+ +- * Filter (133)
+ +- * ColumnarToRow (132)
+ +- Scan parquet spark_catalog.default.date_dim (131)
-(143) Scan parquet spark_catalog.default.date_dim
+(131) Scan parquet spark_catalog.default.date_dim
Output [3]: [d_date_sk#36, d_year#136, d_moy#137]
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,11), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
-(144) ColumnarToRow [codegen id : 1]
+(132) ColumnarToRow [codegen id : 1]
Input [3]: [d_date_sk#36, d_year#136, d_moy#137]
-(145) Filter [codegen id : 1]
+(133) Filter [codegen id : 1]
Input [3]: [d_date_sk#36, d_year#136, d_moy#137]
Condition : ((((isnotnull(d_year#136) AND isnotnull(d_moy#137)) AND (d_year#136 = 2001)) AND (d_moy#137 = 11)) AND isnotnull(d_date_sk#36))
-(146) Project [codegen id : 1]
+(134) Project [codegen id : 1]
Output [1]: [d_date_sk#36]
Input [3]: [d_date_sk#36, d_year#136, d_moy#137]
-(147) BroadcastExchange
+(135) BroadcastExchange
Input [1]: [d_date_sk#36]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16]
-Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
-BroadcastExchange (152)
-+- * Project (151)
- +- * Filter (150)
- +- * ColumnarToRow (149)
- +- Scan parquet spark_catalog.default.date_dim (148)
+Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
+BroadcastExchange (140)
++- * Project (139)
+ +- * Filter (138)
+ +- * ColumnarToRow (137)
+ +- Scan parquet spark_catalog.default.date_dim (136)
-(148) Scan parquet spark_catalog.default.date_dim
+(136) Scan parquet spark_catalog.default.date_dim
Output [2]: [d_date_sk#13, d_year#138]
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<d_date_sk:int,d_year:int>
-(149) ColumnarToRow [codegen id : 1]
+(137) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#13, d_year#138]
-(150) Filter [codegen id : 1]
+(138) Filter [codegen id : 1]
Input [2]: [d_date_sk#13, d_year#138]
Condition : (((isnotnull(d_year#138) AND (d_year#138 >= 1999)) AND (d_year#138 <= 2001)) AND isnotnull(d_date_sk#13))
-(151) Project [codegen id : 1]
+(139) Project [codegen id : 1]
Output [1]: [d_date_sk#13]
Input [2]: [d_date_sk#13, d_year#138]
-(152) BroadcastExchange
+(140) BroadcastExchange
Input [1]: [d_date_sk#13]
-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=17]
-Subquery:7 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
+Subquery:7 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
-Subquery:8 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
+Subquery:8 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
-Subquery:9 Hosting operator id = 97 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52]
+Subquery:9 Hosting operator id = 88 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52]
-Subquery:10 Hosting operator id = 80 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5
+Subquery:10 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5
-Subquery:11 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52]
+Subquery:11 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52]
-Subquery:12 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5
+Subquery:12 Hosting operator id = 90 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt
index 460d252e030..7a7ce7f20b5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt
@@ -1,21 +1,21 @@
TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)]
- WholeStageCodegen (134)
+ WholeStageCodegen (116)
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum]
InputAdapter
Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1
- WholeStageCodegen (133)
+ WholeStageCodegen (115)
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
InputAdapter
Union
- WholeStageCodegen (44)
+ WholeStageCodegen (38)
Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
Filter [sales]
Subquery #3
WholeStageCodegen (8)
HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
InputAdapter
- Exchange #17
+ Exchange #15
WholeStageCodegen (7)
HashAggregate [quantity,list_price] [sum,count,sum,count]
InputAdapter
@@ -28,7 +28,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
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
+ ReusedExchange [d_date_sk] #8
WholeStageCodegen (4)
Project [cs_quantity,cs_list_price]
BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
@@ -37,7 +37,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #2
InputAdapter
- ReusedExchange [d_date_sk] #9
+ ReusedExchange [d_date_sk] #8
WholeStageCodegen (6)
Project [ws_quantity,ws_list_price]
BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
@@ -46,223 +46,187 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #2
InputAdapter
- ReusedExchange [d_date_sk] #9
+ ReusedExchange [d_date_sk] #8
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),sales,number_sales,sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #2
- WholeStageCodegen (43)
+ 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]
- SortMergeJoin [ss_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (2)
- Sort [ss_item_sk]
- InputAdapter
- Exchange [ss_item_sk] #3
- WholeStageCodegen (1)
- 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]
- InputAdapter
- WholeStageCodegen (20)
- Sort [ss_item_sk]
- InputAdapter
- Exchange [ss_item_sk] #5
- WholeStageCodegen (19)
- 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]
+ 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 (18)
- 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 (13)
- Sort [brand_id,class_id,category_id]
- InputAdapter
- Exchange [brand_id,class_id,category_id] #7
- WholeStageCodegen (12)
- HashAggregate [brand_id,class_id,category_id]
- InputAdapter
- Exchange [brand_id,class_id,category_id] #8
- WholeStageCodegen (11)
- 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 (10)
- 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 (5)
- Sort [i_brand_id,i_class_id,i_category_id]
- InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #11
- WholeStageCodegen (4)
- 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 (9)
- Sort [i_brand_id,i_class_id,i_category_id]
- InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #12
- WholeStageCodegen (8)
- 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 (7)
- 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 (17)
- Sort [i_brand_id,i_class_id,i_category_id]
- InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #14
- WholeStageCodegen (16)
- 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
+ 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] #4
+ ReusedExchange [d_date_sk] #3
InputAdapter
- BroadcastExchange #15
- WholeStageCodegen (42)
- SortMergeJoin [i_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (23)
- Sort [i_item_sk]
- InputAdapter
- Exchange [i_item_sk] #16
- WholeStageCodegen (22)
- Filter [i_item_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ BroadcastExchange #14
+ 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
- WholeStageCodegen (41)
- Sort [ss_item_sk]
- InputAdapter
- ReusedExchange [ss_item_sk] #5
- WholeStageCodegen (88)
+ ReusedExchange [ss_item_sk] #4
+ WholeStageCodegen (76)
Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
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),sales,number_sales,sum,isEmpty,count]
InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #18
- WholeStageCodegen (87)
+ Exchange [i_brand_id,i_class_id,i_category_id] #16
+ 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]
- SortMergeJoin [cs_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (46)
- Sort [cs_item_sk]
- InputAdapter
- Exchange [cs_item_sk] #19
- WholeStageCodegen (45)
- 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
+ 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
InputAdapter
- WholeStageCodegen (64)
- Sort [ss_item_sk]
- InputAdapter
- ReusedExchange [ss_item_sk] #5
+ ReusedExchange [ss_item_sk] #4
InputAdapter
- ReusedExchange [d_date_sk] #4
+ ReusedExchange [d_date_sk] #3
InputAdapter
- ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15
- WholeStageCodegen (132)
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14
+ WholeStageCodegen (114)
Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
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),sales,number_sales,sum,isEmpty,count]
InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #20
- WholeStageCodegen (131)
+ Exchange [i_brand_id,i_class_id,i_category_id] #17
+ 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]
- SortMergeJoin [ws_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (90)
- Sort [ws_item_sk]
- InputAdapter
- Exchange [ws_item_sk] #21
- WholeStageCodegen (89)
- 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
+ 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
- WholeStageCodegen (108)
- Sort [ss_item_sk]
- InputAdapter
- ReusedExchange [ss_item_sk] #5
+ ReusedExchange [ss_item_sk] #4
InputAdapter
- ReusedExchange [d_date_sk] #4
+ ReusedExchange [d_date_sk] #3
InputAdapter
- ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt
index 22b9222624d..99573a62028 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt
@@ -1,103 +1,94 @@
== Physical Plan ==
-TakeOrderedAndProject (99)
-+- * BroadcastHashJoin Inner BuildRight (98)
- :- * Filter (78)
- : +- * HashAggregate (77)
- : +- Exchange (76)
- : +- * HashAggregate (75)
- : +- * Project (74)
- : +- * BroadcastHashJoin Inner BuildRight (73)
- : :- * Project (63)
- : : +- * BroadcastHashJoin Inner BuildRight (62)
- : : :- * SortMergeJoin LeftSemi (60)
- : : : :- * Sort (5)
- : : : : +- Exchange (4)
- : : : : +- * Filter (3)
- : : : : +- * ColumnarToRow (2)
- : : : : +- Scan parquet spark_catalog.default.store_sales (1)
- : : : +- * Sort (59)
- : : : +- Exchange (58)
- : : : +- * Project (57)
- : : : +- * BroadcastHashJoin Inner BuildRight (56)
- : : : :- * Filter (8)
- : : : : +- * ColumnarToRow (7)
- : : : : +- Scan parquet spark_catalog.default.item (6)
- : : : +- BroadcastExchange (55)
- : : : +- * SortMergeJoin LeftSemi (54)
- : : : :- * Sort (42)
- : : : : +- Exchange (41)
- : : : : +- * HashAggregate (40)
- : : : : +- Exchange (39)
- : : : : +- * HashAggregate (38)
- : : : : +- * Project (37)
- : : : : +- * BroadcastHashJoin Inner BuildRight (36)
- : : : : :- * Project (14)
- : : : : : +- * BroadcastHashJoin Inner BuildRight (13)
- : : : : : :- * Filter (11)
- : : : : : : +- * ColumnarToRow (10)
- : : : : : : +- Scan parquet spark_catalog.default.store_sales (9)
- : : : : : +- ReusedExchange (12)
- : : : : +- BroadcastExchange (35)
- : : : : +- * SortMergeJoin LeftSemi (34)
- : : : : :- * Sort (19)
- : : : : : +- Exchange (18)
- : : : : : +- * Filter (17)
- : : : : : +- * ColumnarToRow (16)
- : : : : : +- Scan parquet spark_catalog.default.item (15)
- : : : : +- * Sort (33)
- : : : : +- Exchange (32)
- : : : : +- * Project (31)
- : : : : +- * BroadcastHashJoin Inner BuildRight (30)
- : : : : :- * Project (25)
- : : : : : +- * BroadcastHashJoin Inner BuildRight (24)
- : : : : : :- * Filter (22)
- : : : : : : +- * ColumnarToRow (21)
- : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (20)
- : : : : : +- ReusedExchange (23)
- : : : : +- BroadcastExchange (29)
- : : : : +- * Filter (28)
- : : : : +- * ColumnarToRow (27)
- : : : : +- Scan parquet spark_catalog.default.item (26)
- : : : +- * Sort (53)
- : : : +- Exchange (52)
- : : : +- * Project (51)
- : : : +- * BroadcastHashJoin Inner BuildRight (50)
- : : : :- * Project (48)
- : : : : +- * BroadcastHashJoin Inner BuildRight (47)
- : : : : :- * Filter (45)
- : : : : : +- * ColumnarToRow (44)
- : : : : : +- Scan parquet spark_catalog.default.web_sales (43)
- : : : : +- ReusedExchange (46)
- : : : +- ReusedExchange (49)
- : : +- ReusedExchange (61)
- : +- BroadcastExchange (72)
- : +- * SortMergeJoin LeftSemi (71)
- : :- * Sort (68)
- : : +- Exchange (67)
- : : +- * Filter (66)
- : : +- * ColumnarToRow (65)
- : : +- Scan parquet spark_catalog.default.item (64)
- : +- * Sort (70)
- : +- ReusedExchange (69)
- +- BroadcastExchange (97)
- +- * Filter (96)
- +- * HashAggregate (95)
- +- Exchange (94)
- +- * HashAggregate (93)
- +- * Project (92)
- +- * BroadcastHashJoin Inner BuildRight (91)
- :- * Project (89)
- : +- * BroadcastHashJoin Inner BuildRight (88)
- : :- * SortMergeJoin LeftSemi (86)
- : : :- * Sort (83)
- : : : +- Exchange (82)
- : : : +- * Filter (81)
- : : : +- * ColumnarToRow (80)
- : : : +- Scan parquet spark_catalog.default.store_sales (79)
- : : +- * Sort (85)
- : : +- ReusedExchange (84)
- : +- ReusedExchange (87)
- +- ReusedExchange (90)
+TakeOrderedAndProject (90)
++- * BroadcastHashJoin Inner BuildRight (89)
+ :- * 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)
+ +- BroadcastExchange (88)
+ +- * 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.store_sales (73)
+ : : +- ReusedExchange (76)
+ : +- ReusedExchange (78)
+ +- ReusedExchange (81)
(1) Scan parquet spark_catalog.default.store_sales
@@ -108,36 +99,28 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol
PushedFilters: [IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)>
-(2) ColumnarToRow [codegen id : 1]
+(2) ColumnarToRow [codegen id : 37]
Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-(3) Filter [codegen id : 1]
+(3) Filter [codegen id : 37]
Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
Condition : isnotnull(ss_item_sk#1)
-(4) Exchange
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1]
-
-(5) Sort [codegen id : 2]
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0
-
-(6) Scan parquet spark_catalog.default.item
+(4) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(7) ColumnarToRow [codegen id : 19]
+(5) ColumnarToRow [codegen id : 17]
Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
-(8) Filter [codegen id : 19]
+(6) Filter [codegen id : 17]
Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9))
-(9) Scan parquet spark_catalog.default.store_sales
+(7) Scan parquet spark_catalog.default.store_sales
Output [2]: [ss_item_sk#10, ss_sold_date_sk#11]
Batched: true
Location: InMemoryFileIndex []
@@ -145,49 +128,49 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so
PushedFilters: [IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int>
-(10) ColumnarToRow [codegen id : 11]
+(8) ColumnarToRow [codegen id : 9]
Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
-(11) Filter [codegen id : 11]
+(9) Filter [codegen id : 9]
Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
Condition : isnotnull(ss_item_sk#10)
-(12) ReusedExchange [Reuses operator id: 132]
+(10) ReusedExchange [Reuses operator id: 123]
Output [1]: [d_date_sk#13]
-(13) BroadcastHashJoin [codegen id : 11]
+(11) BroadcastHashJoin [codegen id : 9]
Left keys [1]: [ss_sold_date_sk#11]
Right keys [1]: [d_date_sk#13]
Join type: Inner
Join condition: None
-(14) Project [codegen id : 11]
+(12) Project [codegen id : 9]
Output [1]: [ss_item_sk#10]
Input [3]: [ss_item_sk#10, ss_sold_date_sk#11, d_date_sk#13]
-(15) Scan parquet spark_catalog.default.item
+(13) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(16) ColumnarToRow [codegen id : 4]
+(14) ColumnarToRow [codegen id : 2]
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-(17) Filter [codegen id : 4]
+(15) Filter [codegen id : 2]
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
Condition : (((isnotnull(i_item_sk#14) AND isnotnull(i_brand_id#15)) AND isnotnull(i_class_id#16)) AND isnotnull(i_category_id#17))
-(18) Exchange
+(16) Exchange
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=2]
+Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=1]
-(19) Sort [codegen id : 5]
+(17) Sort [codegen id : 3]
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
Arguments: [coalesce(i_brand_id#15, 0) ASC NULLS FIRST, isnull(i_brand_id#15) ASC NULLS FIRST, coalesce(i_class_id#16, 0) ASC NULLS FIRST, isnull(i_class_id#16) ASC NULLS FIRST, coalesce(i_category_id#17, 0) ASC NULLS FIRST, isnull(i_category_id#17) ASC NULLS FIRST], false, 0
-(20) Scan parquet spark_catalog.default.catalog_sales
+(18) Scan parquet spark_catalog.default.catalog_sales
Output [2]: [cs_item_sk#18, cs_sold_date_sk#19]
Batched: true
Location: InMemoryFileIndex []
@@ -195,109 +178,109 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_so
PushedFilters: [IsNotNull(cs_item_sk)]
ReadSchema: struct<cs_item_sk:int>
-(21) ColumnarToRow [codegen id : 8]
+(19) ColumnarToRow [codegen id : 6]
Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
-(22) Filter [codegen id : 8]
+(20) Filter [codegen id : 6]
Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
Condition : isnotnull(cs_item_sk#18)
-(23) ReusedExchange [Reuses operator id: 132]
+(21) ReusedExchange [Reuses operator id: 123]
Output [1]: [d_date_sk#20]
-(24) BroadcastHashJoin [codegen id : 8]
+(22) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [cs_sold_date_sk#19]
Right keys [1]: [d_date_sk#20]
Join type: Inner
Join condition: None
-(25) Project [codegen id : 8]
+(23) Project [codegen id : 6]
Output [1]: [cs_item_sk#18]
Input [3]: [cs_item_sk#18, cs_sold_date_sk#19, d_date_sk#20]
-(26) Scan parquet spark_catalog.default.item
+(24) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(27) ColumnarToRow [codegen id : 7]
+(25) ColumnarToRow [codegen id : 5]
Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-(28) Filter [codegen id : 7]
+(26) Filter [codegen id : 5]
Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
Condition : isnotnull(i_item_sk#21)
-(29) BroadcastExchange
+(27) BroadcastExchange
Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2]
-(30) BroadcastHashJoin [codegen id : 8]
+(28) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [cs_item_sk#18]
Right keys [1]: [i_item_sk#21]
Join type: Inner
Join condition: None
-(31) Project [codegen id : 8]
+(29) Project [codegen id : 6]
Output [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
Input [5]: [cs_item_sk#18, i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-(32) Exchange
+(30) Exchange
Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=4]
+Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=3]
-(33) Sort [codegen id : 9]
+(31) Sort [codegen id : 7]
Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
Arguments: [coalesce(i_brand_id#22, 0) ASC NULLS FIRST, isnull(i_brand_id#22) ASC NULLS FIRST, coalesce(i_class_id#23, 0) ASC NULLS FIRST, isnull(i_class_id#23) ASC NULLS FIRST, coalesce(i_category_id#24, 0) ASC NULLS FIRST, isnull(i_category_id#24) ASC NULLS FIRST], false, 0
-(34) SortMergeJoin [codegen id : 10]
+(32) SortMergeJoin [codegen id : 8]
Left keys [6]: [coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17)]
Right keys [6]: [coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24)]
Join type: LeftSemi
Join condition: None
-(35) BroadcastExchange
+(33) BroadcastExchange
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
-(36) BroadcastHashJoin [codegen id : 11]
+(34) BroadcastHashJoin [codegen id : 9]
Left keys [1]: [ss_item_sk#10]
Right keys [1]: [i_item_sk#14]
Join type: Inner
Join condition: None
-(37) Project [codegen id : 11]
+(35) Project [codegen id : 9]
Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27]
Input [5]: [ss_item_sk#10, i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-(38) HashAggregate [codegen id : 11]
+(36) HashAggregate [codegen id : 9]
Input [3]: [brand_id#25, class_id#26, category_id#27]
Keys [3]: [brand_id#25, class_id#26, category_id#27]
Functions: []
Aggregate Attributes: []
Results [3]: [brand_id#25, class_id#26, category_id#27]
-(39) Exchange
+(37) Exchange
Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=5]
-(40) HashAggregate [codegen id : 12]
+(38) HashAggregate [codegen id : 10]
Input [3]: [brand_id#25, class_id#26, category_id#27]
Keys [3]: [brand_id#25, class_id#26, category_id#27]
Functions: []
Aggregate Attributes: []
Results [3]: [brand_id#25, class_id#26, category_id#27]
-(41) Exchange
+(39) Exchange
Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=7]
+Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=6]
-(42) Sort [codegen id : 13]
+(40) Sort [codegen id : 11]
Input [3]: [brand_id#25, class_id#26, category_id#27]
Arguments: [coalesce(brand_id#25, 0) ASC NULLS FIRST, isnull(brand_id#25) ASC NULLS FIRST, coalesce(class_id#26, 0) ASC NULLS FIRST, isnull(class_id#26) ASC NULLS FIRST, coalesce(category_id#27, 0) ASC NULLS FIRST, isnull(category_id#27) ASC NULLS FIRST], false, 0
-(43) Scan parquet spark_catalog.default.web_sales
+(41) Scan parquet spark_catalog.default.web_sales
Output [2]: [ws_item_sk#28, ws_sold_date_sk#29]
Batched: true
Location: InMemoryFileIndex []
@@ -305,166 +288,150 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so
PushedFilters: [IsNotNull(ws_item_sk)]
ReadSchema: struct<ws_item_sk:int>
-(44) ColumnarToRow [codegen id : 16]
+(42) ColumnarToRow [codegen id : 14]
Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
-(45) Filter [codegen id : 16]
+(43) Filter [codegen id : 14]
Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
Condition : isnotnull(ws_item_sk#28)
-(46) ReusedExchange [Reuses operator id: 132]
+(44) ReusedExchange [Reuses operator id: 123]
Output [1]: [d_date_sk#30]
-(47) BroadcastHashJoin [codegen id : 16]
+(45) BroadcastHashJoin [codegen id : 14]
Left keys [1]: [ws_sold_date_sk#29]
Right keys [1]: [d_date_sk#30]
Join type: Inner
Join condition: None
-(48) Project [codegen id : 16]
+(46) Project [codegen id : 14]
Output [1]: [ws_item_sk#28]
Input [3]: [ws_item_sk#28, ws_sold_date_sk#29, d_date_sk#30]
-(49) ReusedExchange [Reuses operator id: 29]
+(47) ReusedExchange [Reuses operator id: 27]
Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
-(50) BroadcastHashJoin [codegen id : 16]
+(48) BroadcastHashJoin [codegen id : 14]
Left keys [1]: [ws_item_sk#28]
Right keys [1]: [i_item_sk#31]
Join type: Inner
Join condition: None
-(51) Project [codegen id : 16]
+(49) Project [codegen id : 14]
Output [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
Input [5]: [ws_item_sk#28, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
-(52) Exchange
+(50) Exchange
Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
-Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=8]
+Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=7]
-(53) Sort [codegen id : 17]
+(51) Sort [codegen id : 15]
Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
Arguments: [coalesce(i_brand_id#32, 0) ASC NULLS FIRST, isnull(i_brand_id#32) ASC NULLS FIRST, coalesce(i_class_id#33, 0) ASC NULLS FIRST, isnull(i_class_id#33) ASC NULLS FIRST, coalesce(i_category_id#34, 0) ASC NULLS FIRST, isnull(i_category_id#34) ASC NULLS FIRST], false, 0
-(54) SortMergeJoin [codegen id : 18]
+(52) SortMergeJoin [codegen id : 16]
Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)]
Right keys [6]: [coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34)]
Join type: LeftSemi
Join condition: None
-(55) BroadcastExchange
+(53) BroadcastExchange
Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=9]
+Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=8]
-(56) BroadcastHashJoin [codegen id : 19]
+(54) BroadcastHashJoin [codegen id : 17]
Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Right keys [3]: [brand_id#25, class_id#26, category_id#27]
Join type: Inner
Join condition: None
-(57) Project [codegen id : 19]
+(55) Project [codegen id : 17]
Output [1]: [i_item_sk#6 AS ss_item_sk#35]
Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27]
-(58) Exchange
+(56) BroadcastExchange
Input [1]: [ss_item_sk#35]
-Arguments: hashpartitioning(ss_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=10]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9]
-(59) Sort [codegen id : 20]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(60) SortMergeJoin [codegen id : 43]
+(57) BroadcastHashJoin [codegen id : 37]
Left keys [1]: [ss_item_sk#1]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(61) ReusedExchange [Reuses operator id: 123]
+(58) ReusedExchange [Reuses operator id: 114]
Output [1]: [d_date_sk#36]
-(62) BroadcastHashJoin [codegen id : 43]
+(59) BroadcastHashJoin [codegen id : 37]
Left keys [1]: [ss_sold_date_sk#4]
Right keys [1]: [d_date_sk#36]
Join type: Inner
Join condition: None
-(63) Project [codegen id : 43]
+(60) Project [codegen id : 37]
Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3]
Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36]
-(64) Scan parquet spark_catalog.default.item
+(61) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(65) ColumnarToRow [codegen id : 22]
+(62) ColumnarToRow [codegen id : 36]
Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-(66) Filter [codegen id : 22]
+(63) Filter [codegen id : 36]
Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
Condition : (((isnotnull(i_item_sk#37) AND isnotnull(i_brand_id#38)) AND isnotnull(i_class_id#39)) AND isnotnull(i_category_id#40))
-(67) Exchange
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: hashpartitioning(i_item_sk#37, 5), ENSURE_REQUIREMENTS, [plan_id=11]
-
-(68) Sort [codegen id : 23]
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: [i_item_sk#37 ASC NULLS FIRST], false, 0
-
-(69) ReusedExchange [Reuses operator id: 58]
+(64) ReusedExchange [Reuses operator id: 56]
Output [1]: [ss_item_sk#35]
-(70) Sort [codegen id : 41]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(71) SortMergeJoin [codegen id : 42]
+(65) BroadcastHashJoin [codegen id : 36]
Left keys [1]: [i_item_sk#37]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(72) BroadcastExchange
+(66) BroadcastExchange
Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10]
-(73) BroadcastHashJoin [codegen id : 43]
+(67) BroadcastHashJoin [codegen id : 37]
Left keys [1]: [ss_item_sk#1]
Right keys [1]: [i_item_sk#37]
Join type: Inner
Join condition: None
-(74) Project [codegen id : 43]
+(68) Project [codegen id : 37]
Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-(75) HashAggregate [codegen id : 43]
+(69) HashAggregate [codegen id : 37]
Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
-(76) Exchange
+(70) Exchange
Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
-Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=13]
+Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11]
-(77) HashAggregate [codegen id : 88]
+(71) HashAggregate [codegen id : 76]
Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51]
-(78) Filter [codegen id : 88]
+(72) Filter [codegen id : 76]
Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51]
Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
-(79) Scan parquet spark_catalog.default.store_sales
+(73) Scan parquet spark_catalog.default.store_sales
Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
Batched: true
Location: InMemoryFileIndex []
@@ -472,358 +439,346 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so
PushedFilters: [IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)>
-(80) ColumnarToRow [codegen id : 44]
+(74) ColumnarToRow [codegen id : 74]
Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
-(81) Filter [codegen id : 44]
+(75) Filter [codegen id : 74]
Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
Condition : isnotnull(ss_item_sk#54)
-(82) Exchange
-Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
-Arguments: hashpartitioning(ss_item_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=14]
-
-(83) Sort [codegen id : 45]
-Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
-Arguments: [ss_item_sk#54 ASC NULLS FIRST], false, 0
-
-(84) ReusedExchange [Reuses operator id: 58]
+(76) ReusedExchange [Reuses operator id: 56]
Output [1]: [ss_item_sk#35]
-(85) Sort [codegen id : 63]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(86) SortMergeJoin [codegen id : 86]
+(77) BroadcastHashJoin [codegen id : 74]
Left keys [1]: [ss_item_sk#54]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(87) ReusedExchange [Reuses operator id: 137]
+(78) ReusedExchange [Reuses operator id: 128]
Output [1]: [d_date_sk#59]
-(88) BroadcastHashJoin [codegen id : 86]
+(79) BroadcastHashJoin [codegen id : 74]
Left keys [1]: [ss_sold_date_sk#57]
Right keys [1]: [d_date_sk#59]
Join type: Inner
Join condition: None
-(89) Project [codegen id : 86]
+(80) Project [codegen id : 74]
Output [3]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56]
Input [5]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, d_date_sk#59]
-(90) ReusedExchange [Reuses operator id: 72]
+(81) ReusedExchange [Reuses operator id: 66]
Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63]
-(91) BroadcastHashJoin [codegen id : 86]
+(82) BroadcastHashJoin [codegen id : 74]
Left keys [1]: [ss_item_sk#54]
Right keys [1]: [i_item_sk#60]
Join type: Inner
Join condition: None
-(92) Project [codegen id : 86]
+(83) Project [codegen id : 74]
Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63]
Input [7]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63]
-(93) HashAggregate [codegen id : 86]
+(84) HashAggregate [codegen id : 74]
Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63]
Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)]
Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66]
Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
-(94) Exchange
+(85) Exchange
Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
-Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=15]
+Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=12]
-(95) HashAggregate [codegen id : 87]
+(86) HashAggregate [codegen id : 75]
Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)]
Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71]
Results [6]: [store AS channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74]
-(96) Filter [codegen id : 87]
+(87) Filter [codegen id : 75]
Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
-(97) BroadcastExchange
+(88) BroadcastExchange
Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
-Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=16]
+Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=13]
-(98) BroadcastHashJoin [codegen id : 88]
+(89) BroadcastHashJoin [codegen id : 76]
Left keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
Join type: Inner
Join condition: None
-(99) TakeOrderedAndProject
+(90) TakeOrderedAndProject
Input [12]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
Arguments: 100, [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, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
===== Subqueries =====
-Subquery:1 Hosting operator id = 78 Hosting Expression = Subquery scalar-subquery#52, [id=#53]
-* HashAggregate (118)
-+- Exchange (117)
- +- * HashAggregate (116)
- +- Union (115)
- :- * Project (104)
- : +- * BroadcastHashJoin Inner BuildRight (103)
- : :- * ColumnarToRow (101)
- : : +- Scan parquet spark_catalog.default.store_sales (100)
- : +- ReusedExchange (102)
- :- * Project (109)
- : +- * BroadcastHashJoin Inner BuildRight (108)
- : :- * ColumnarToRow (106)
- : : +- Scan parquet spark_catalog.default.catalog_sales (105)
- : +- ReusedExchange (107)
- +- * Project (114)
- +- * BroadcastHashJoin Inner BuildRight (113)
- :- * ColumnarToRow (111)
- : +- Scan parquet spark_catalog.default.web_sales (110)
- +- ReusedExchange (112)
-
-
-(100) Scan parquet spark_catalog.default.store_sales
+Subquery:1 Hosting operator id = 72 Hosting Expression = Subquery scalar-subquery#52, [id=#53]
+* HashAggregate (109)
++- Exchange (108)
+ +- * HashAggregate (107)
+ +- Union (106)
+ :- * Project (95)
+ : +- * BroadcastHashJoin Inner BuildRight (94)
+ : :- * ColumnarToRow (92)
+ : : +- Scan parquet spark_catalog.default.store_sales (91)
+ : +- ReusedExchange (93)
+ :- * Project (100)
+ : +- * BroadcastHashJoin Inner BuildRight (99)
+ : :- * ColumnarToRow (97)
+ : : +- Scan parquet spark_catalog.default.catalog_sales (96)
+ : +- ReusedExchange (98)
+ +- * Project (105)
+ +- * BroadcastHashJoin Inner BuildRight (104)
+ :- * ColumnarToRow (102)
+ : +- Scan parquet spark_catalog.default.web_sales (101)
+ +- ReusedExchange (103)
+
+
+(91) Scan parquet spark_catalog.default.store_sales
Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)]
ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)>
-(101) ColumnarToRow [codegen id : 2]
+(92) ColumnarToRow [codegen id : 2]
Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77]
-(102) ReusedExchange [Reuses operator id: 132]
+(93) ReusedExchange [Reuses operator id: 123]
Output [1]: [d_date_sk#78]
-(103) BroadcastHashJoin [codegen id : 2]
+(94) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [ss_sold_date_sk#77]
Right keys [1]: [d_date_sk#78]
Join type: Inner
Join condition: None
-(104) Project [codegen id : 2]
+(95) Project [codegen id : 2]
Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80]
Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78]
-(105) Scan parquet spark_catalog.default.catalog_sales
+(96) Scan parquet spark_catalog.default.catalog_sales
Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)]
ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)>
-(106) ColumnarToRow [codegen id : 4]
+(97) ColumnarToRow [codegen id : 4]
Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83]
-(107) ReusedExchange [Reuses operator id: 132]
+(98) ReusedExchange [Reuses operator id: 123]
Output [1]: [d_date_sk#84]
-(108) BroadcastHashJoin [codegen id : 4]
+(99) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [cs_sold_date_sk#83]
Right keys [1]: [d_date_sk#84]
Join type: Inner
Join condition: None
-(109) Project [codegen id : 4]
+(100) Project [codegen id : 4]
Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86]
Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84]
-(110) Scan parquet spark_catalog.default.web_sales
+(101) Scan parquet spark_catalog.default.web_sales
Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)]
ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)>
-(111) ColumnarToRow [codegen id : 6]
+(102) ColumnarToRow [codegen id : 6]
Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89]
-(112) ReusedExchange [Reuses operator id: 132]
+(103) ReusedExchange [Reuses operator id: 123]
Output [1]: [d_date_sk#90]
-(113) BroadcastHashJoin [codegen id : 6]
+(104) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [ws_sold_date_sk#89]
Right keys [1]: [d_date_sk#90]
Join type: Inner
Join condition: None
-(114) Project [codegen id : 6]
+(105) Project [codegen id : 6]
Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92]
Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90]
-(115) Union
+(106) Union
-(116) HashAggregate [codegen id : 7]
+(107) HashAggregate [codegen id : 7]
Input [2]: [quantity#79, list_price#80]
Keys: []
Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
Aggregate Attributes [2]: [sum#93, count#94]
Results [2]: [sum#95, count#96]
-(117) Exchange
+(108) Exchange
Input [2]: [sum#95, count#96]
-Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14]
-(118) HashAggregate [codegen id : 8]
+(109) HashAggregate [codegen id : 8]
Input [2]: [sum#95, count#96]
Keys: []
Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97]
Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98]
-Subquery:2 Hosting operator id = 100 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12
+Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12
-Subquery:3 Hosting operator id = 105 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12
+Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12
-Subquery:4 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12
+Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12
Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (123)
-+- * Project (122)
- +- * Filter (121)
- +- * ColumnarToRow (120)
- +- Scan parquet spark_catalog.default.date_dim (119)
+BroadcastExchange (114)
++- * Project (113)
+ +- * Filter (112)
+ +- * ColumnarToRow (111)
+ +- Scan parquet spark_catalog.default.date_dim (110)
-(119) Scan parquet spark_catalog.default.date_dim
+(110) Scan parquet spark_catalog.default.date_dim
Output [2]: [d_date_sk#36, d_week_seq#99]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_week_seq:int>
-(120) ColumnarToRow [codegen id : 1]
+(111) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#36, d_week_seq#99]
-(121) Filter [codegen id : 1]
+(112) Filter [codegen id : 1]
Input [2]: [d_date_sk#36, d_week_seq#99]
Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#36))
-(122) Project [codegen id : 1]
+(113) Project [codegen id : 1]
Output [1]: [d_date_sk#36]
Input [2]: [d_date_sk#36, d_week_seq#99]
-(123) BroadcastExchange
+(114) BroadcastExchange
Input [1]: [d_date_sk#36]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=18]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15]
-Subquery:6 Hosting operator id = 121 Hosting Expression = Subquery scalar-subquery#100, [id=#101]
-* Project (127)
-+- * Filter (126)
- +- * ColumnarToRow (125)
- +- Scan parquet spark_catalog.default.date_dim (124)
+Subquery:6 Hosting operator id = 112 Hosting Expression = Subquery scalar-subquery#100, [id=#101]
+* Project (118)
++- * Filter (117)
+ +- * ColumnarToRow (116)
+ +- Scan parquet spark_catalog.default.date_dim (115)
-(124) Scan parquet spark_catalog.default.date_dim
+(115) Scan parquet spark_catalog.default.date_dim
Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)]
ReadSchema: struct<d_week_seq:int,d_year:int,d_moy:int,d_dom:int>
-(125) ColumnarToRow [codegen id : 1]
+(116) ColumnarToRow [codegen id : 1]
Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
-(126) Filter [codegen id : 1]
+(117) Filter [codegen id : 1]
Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 2000)) AND (d_moy#104 = 12)) AND (d_dom#105 = 11))
-(127) Project [codegen id : 1]
+(118) Project [codegen id : 1]
Output [1]: [d_week_seq#102]
Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
-Subquery:7 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
-BroadcastExchange (132)
-+- * Project (131)
- +- * Filter (130)
- +- * ColumnarToRow (129)
- +- Scan parquet spark_catalog.default.date_dim (128)
+Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
+BroadcastExchange (123)
++- * Project (122)
+ +- * Filter (121)
+ +- * ColumnarToRow (120)
+ +- Scan parquet spark_catalog.default.date_dim (119)
-(128) Scan parquet spark_catalog.default.date_dim
+(119) Scan parquet spark_catalog.default.date_dim
Output [2]: [d_date_sk#13, d_year#106]
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<d_date_sk:int,d_year:int>
-(129) ColumnarToRow [codegen id : 1]
+(120) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#13, d_year#106]
-(130) Filter [codegen id : 1]
+(121) Filter [codegen id : 1]
Input [2]: [d_date_sk#13, d_year#106]
Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1999)) AND (d_year#106 <= 2001)) AND isnotnull(d_date_sk#13))
-(131) Project [codegen id : 1]
+(122) Project [codegen id : 1]
Output [1]: [d_date_sk#13]
Input [2]: [d_date_sk#13, d_year#106]
-(132) BroadcastExchange
+(123) BroadcastExchange
Input [1]: [d_date_sk#13]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16]
-Subquery:8 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
+Subquery:8 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
-Subquery:9 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
+Subquery:9 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
-Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
+Subquery:10 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
-Subquery:11 Hosting operator id = 79 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58
-BroadcastExchange (137)
-+- * Project (136)
- +- * Filter (135)
- +- * ColumnarToRow (134)
- +- Scan parquet spark_catalog.default.date_dim (133)
+Subquery:11 Hosting operator id = 73 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58
+BroadcastExchange (128)
++- * Project (127)
+ +- * Filter (126)
+ +- * ColumnarToRow (125)
+ +- Scan parquet spark_catalog.default.date_dim (124)
-(133) Scan parquet spark_catalog.default.date_dim
+(124) Scan parquet spark_catalog.default.date_dim
Output [2]: [d_date_sk#59, d_week_seq#107]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_week_seq:int>
-(134) ColumnarToRow [codegen id : 1]
+(125) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#59, d_week_seq#107]
-(135) Filter [codegen id : 1]
+(126) Filter [codegen id : 1]
Input [2]: [d_date_sk#59, d_week_seq#107]
Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#59))
-(136) Project [codegen id : 1]
+(127) Project [codegen id : 1]
Output [1]: [d_date_sk#59]
Input [2]: [d_date_sk#59, d_week_seq#107]
-(137) BroadcastExchange
+(128) BroadcastExchange
Input [1]: [d_date_sk#59]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17]
-Subquery:12 Hosting operator id = 135 Hosting Expression = Subquery scalar-subquery#108, [id=#109]
-* Project (141)
-+- * Filter (140)
- +- * ColumnarToRow (139)
- +- Scan parquet spark_catalog.default.date_dim (138)
+Subquery:12 Hosting operator id = 126 Hosting Expression = Subquery scalar-subquery#108, [id=#109]
+* Project (132)
++- * Filter (131)
+ +- * ColumnarToRow (130)
+ +- Scan parquet spark_catalog.default.date_dim (129)
-(138) Scan parquet spark_catalog.default.date_dim
+(129) Scan parquet spark_catalog.default.date_dim
Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)]
ReadSchema: struct<d_week_seq:int,d_year:int,d_moy:int,d_dom:int>
-(139) ColumnarToRow [codegen id : 1]
+(130) ColumnarToRow [codegen id : 1]
Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
-(140) Filter [codegen id : 1]
+(131) Filter [codegen id : 1]
Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1999)) AND (d_moy#112 = 12)) AND (d_dom#113 = 11))
-(141) Project [codegen id : 1]
+(132) Project [codegen id : 1]
Output [1]: [d_week_seq#110]
Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt
index 8c67fff5b14..edd34864986 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt
@@ -1,12 +1,12 @@
TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
- WholeStageCodegen (88)
+ WholeStageCodegen (76)
BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
Filter [sales]
Subquery #4
WholeStageCodegen (8)
HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
InputAdapter
- Exchange #16
+ Exchange #14
WholeStageCodegen (7)
HashAggregate [quantity,list_price] [sum,count,sum,count]
InputAdapter
@@ -19,7 +19,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #3
InputAdapter
- ReusedExchange [d_date_sk] #8
+ ReusedExchange [d_date_sk] #7
WholeStageCodegen (4)
Project [cs_quantity,cs_list_price]
BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
@@ -28,7 +28,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #3
InputAdapter
- ReusedExchange [d_date_sk] #8
+ ReusedExchange [d_date_sk] #7
WholeStageCodegen (6)
Project [ws_quantity,ws_list_price]
BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
@@ -37,211 +37,184 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #3
InputAdapter
- ReusedExchange [d_date_sk] #8
+ ReusedExchange [d_date_sk] #7
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] #1
- WholeStageCodegen (43)
+ 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]
- SortMergeJoin [ss_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (2)
- Sort [ss_item_sk]
- InputAdapter
- Exchange [ss_item_sk] #2
- WholeStageCodegen (1)
- 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_week_seq,d_date_sk]
- Subquery #2
- WholeStageCodegen (1)
- Project [d_week_seq]
- Filter [d_year,d_moy,d_dom]
- ColumnarToRow
- InputAdapter
- Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
- ColumnarToRow
- InputAdapter
- Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
- InputAdapter
- WholeStageCodegen (20)
- Sort [ss_item_sk]
- InputAdapter
- Exchange [ss_item_sk] #4
- WholeStageCodegen (19)
- 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]
+ 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 #2
+ WholeStageCodegen (1)
+ Project [d_date_sk]
+ Filter [d_week_seq,d_date_sk]
+ Subquery #2
+ WholeStageCodegen (1)
+ Project [d_week_seq]
+ Filter [d_year,d_moy,d_dom]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
ColumnarToRow
InputAdapter
- Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
- InputAdapter
- BroadcastExchange #5
- WholeStageCodegen (18)
- 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_week_seq]
+ InputAdapter
+ BroadcastExchange #3
+ 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 #4
+ 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 (13)
- Sort [brand_id,class_id,category_id]
- InputAdapter
- Exchange [brand_id,class_id,category_id] #6
- WholeStageCodegen (12)
- HashAggregate [brand_id,class_id,category_id]
- InputAdapter
- Exchange [brand_id,class_id,category_id] #7
- WholeStageCodegen (11)
- 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] #3
- 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 (10)
- 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] #5
+ WholeStageCodegen (10)
+ HashAggregate [brand_id,class_id,category_id]
+ InputAdapter
+ Exchange [brand_id,class_id,category_id] #6
+ 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] #3
+ 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
+ ReusedExchange [d_date_sk] #7
+ InputAdapter
+ BroadcastExchange #8
+ 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 (5)
- Sort [i_brand_id,i_class_id,i_category_id]
- InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #10
- WholeStageCodegen (4)
- 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] #9
+ 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 (9)
- Sort [i_brand_id,i_class_id,i_category_id]
- InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #11
- WholeStageCodegen (8)
- 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] #3
+ Exchange [i_brand_id,i_class_id,i_category_id] #10
+ 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] #3
+ InputAdapter
+ ReusedExchange [d_date_sk] #7
+ InputAdapter
+ BroadcastExchange #11
+ WholeStageCodegen (5)
+ Filter [i_item_sk]
+ ColumnarToRow
InputAdapter
- ReusedExchange [d_date_sk] #8
- InputAdapter
- BroadcastExchange #12
- WholeStageCodegen (7)
- 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 (17)
- Sort [i_brand_id,i_class_id,i_category_id]
- InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #13
- WholeStageCodegen (16)
- 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] #3
- InputAdapter
- ReusedExchange [d_date_sk] #8
- InputAdapter
- ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12
+ Exchange [i_brand_id,i_class_id,i_category_id] #12
+ 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] #3
+ InputAdapter
+ ReusedExchange [d_date_sk] #7
+ InputAdapter
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11
InputAdapter
- ReusedExchange [d_date_sk] #3
+ ReusedExchange [d_date_sk] #2
InputAdapter
- BroadcastExchange #14
- WholeStageCodegen (42)
- SortMergeJoin [i_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (23)
- Sort [i_item_sk]
- InputAdapter
- Exchange [i_item_sk] #15
- WholeStageCodegen (22)
- 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]
+ BroadcastExchange #13
+ WholeStageCodegen (36)
+ BroadcastHashJoin [i_item_sk,ss_item_sk]
+ 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 (41)
- Sort [ss_item_sk]
- InputAdapter
- ReusedExchange [ss_item_sk] #4
+ ReusedExchange [ss_item_sk] #3
InputAdapter
- BroadcastExchange #17
- WholeStageCodegen (87)
+ BroadcastExchange #15
+ WholeStageCodegen (75)
Filter [sales]
ReusedSubquery [average_sales] #4
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] #18
- WholeStageCodegen (86)
+ Exchange [i_brand_id,i_class_id,i_category_id] #16
+ WholeStageCodegen (74)
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]
- SortMergeJoin [ss_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (45)
- Sort [ss_item_sk]
- InputAdapter
- Exchange [ss_item_sk] #19
- WholeStageCodegen (44)
- 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] #5
- BroadcastExchange #20
- WholeStageCodegen (1)
- Project [d_date_sk]
- Filter [d_week_seq,d_date_sk]
- Subquery #6
- WholeStageCodegen (1)
- Project [d_week_seq]
- Filter [d_year,d_moy,d_dom]
- ColumnarToRow
- InputAdapter
- Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
- ColumnarToRow
- InputAdapter
- Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
+ 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] #5
+ BroadcastExchange #17
+ WholeStageCodegen (1)
+ Project [d_date_sk]
+ Filter [d_week_seq,d_date_sk]
+ Subquery #6
+ WholeStageCodegen (1)
+ Project [d_week_seq]
+ Filter [d_year,d_moy,d_dom]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
InputAdapter
- WholeStageCodegen (63)
- Sort [ss_item_sk]
- InputAdapter
- ReusedExchange [ss_item_sk] #4
+ ReusedExchange [ss_item_sk] #3
InputAdapter
- ReusedExchange [d_date_sk] #20
+ ReusedExchange [d_date_sk] #17
InputAdapter
- ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt
index c87645f3ad8..d7f8bfd6a12 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt
@@ -1,103 +1,94 @@
== Physical Plan ==
-TakeOrderedAndProject (99)
-+- * BroadcastHashJoin Inner BuildRight (98)
- :- * Filter (78)
- : +- * HashAggregate (77)
- : +- Exchange (76)
- : +- * HashAggregate (75)
- : +- * Project (74)
- : +- * BroadcastHashJoin Inner BuildRight (73)
- : :- * Project (63)
- : : +- * BroadcastHashJoin Inner BuildRight (62)
- : : :- * SortMergeJoin LeftSemi (60)
- : : : :- * Sort (5)
- : : : : +- Exchange (4)
- : : : : +- * Filter (3)
- : : : : +- * ColumnarToRow (2)
- : : : : +- Scan parquet spark_catalog.default.store_sales (1)
- : : : +- * Sort (59)
- : : : +- Exchange (58)
- : : : +- * Project (57)
- : : : +- * BroadcastHashJoin Inner BuildRight (56)
- : : : :- * Filter (8)
- : : : : +- * ColumnarToRow (7)
- : : : : +- Scan parquet spark_catalog.default.item (6)
- : : : +- BroadcastExchange (55)
- : : : +- * SortMergeJoin LeftSemi (54)
- : : : :- * Sort (42)
- : : : : +- Exchange (41)
- : : : : +- * HashAggregate (40)
- : : : : +- Exchange (39)
- : : : : +- * HashAggregate (38)
- : : : : +- * Project (37)
- : : : : +- * BroadcastHashJoin Inner BuildRight (36)
- : : : : :- * Project (14)
- : : : : : +- * BroadcastHashJoin Inner BuildRight (13)
- : : : : : :- * Filter (11)
- : : : : : : +- * ColumnarToRow (10)
- : : : : : : +- Scan parquet spark_catalog.default.store_sales (9)
- : : : : : +- ReusedExchange (12)
- : : : : +- BroadcastExchange (35)
- : : : : +- * SortMergeJoin LeftSemi (34)
- : : : : :- * Sort (19)
- : : : : : +- Exchange (18)
- : : : : : +- * Filter (17)
- : : : : : +- * ColumnarToRow (16)
- : : : : : +- Scan parquet spark_catalog.default.item (15)
- : : : : +- * Sort (33)
- : : : : +- Exchange (32)
- : : : : +- * Project (31)
- : : : : +- * BroadcastHashJoin Inner BuildRight (30)
- : : : : :- * Project (25)
- : : : : : +- * BroadcastHashJoin Inner BuildRight (24)
- : : : : : :- * Filter (22)
- : : : : : : +- * ColumnarToRow (21)
- : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (20)
- : : : : : +- ReusedExchange (23)
- : : : : +- BroadcastExchange (29)
- : : : : +- * Filter (28)
- : : : : +- * ColumnarToRow (27)
- : : : : +- Scan parquet spark_catalog.default.item (26)
- : : : +- * Sort (53)
- : : : +- Exchange (52)
- : : : +- * Project (51)
- : : : +- * BroadcastHashJoin Inner BuildRight (50)
- : : : :- * Project (48)
- : : : : +- * BroadcastHashJoin Inner BuildRight (47)
- : : : : :- * Filter (45)
- : : : : : +- * ColumnarToRow (44)
- : : : : : +- Scan parquet spark_catalog.default.web_sales (43)
- : : : : +- ReusedExchange (46)
- : : : +- ReusedExchange (49)
- : : +- ReusedExchange (61)
- : +- BroadcastExchange (72)
- : +- * SortMergeJoin LeftSemi (71)
- : :- * Sort (68)
- : : +- Exchange (67)
- : : +- * Filter (66)
- : : +- * ColumnarToRow (65)
- : : +- Scan parquet spark_catalog.default.item (64)
- : +- * Sort (70)
- : +- ReusedExchange (69)
- +- BroadcastExchange (97)
- +- * Filter (96)
- +- * HashAggregate (95)
- +- Exchange (94)
- +- * HashAggregate (93)
- +- * Project (92)
- +- * BroadcastHashJoin Inner BuildRight (91)
- :- * Project (89)
- : +- * BroadcastHashJoin Inner BuildRight (88)
- : :- * SortMergeJoin LeftSemi (86)
- : : :- * Sort (83)
- : : : +- Exchange (82)
- : : : +- * Filter (81)
- : : : +- * ColumnarToRow (80)
- : : : +- Scan parquet spark_catalog.default.store_sales (79)
- : : +- * Sort (85)
- : : +- ReusedExchange (84)
- : +- ReusedExchange (87)
- +- ReusedExchange (90)
+TakeOrderedAndProject (90)
++- * BroadcastHashJoin Inner BuildRight (89)
+ :- * 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)
+ +- BroadcastExchange (88)
+ +- * 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.store_sales (73)
+ : : +- ReusedExchange (76)
+ : +- ReusedExchange (78)
+ +- ReusedExchange (81)
(1) Scan parquet spark_catalog.default.store_sales
@@ -108,36 +99,28 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol
PushedFilters: [IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)>
-(2) ColumnarToRow [codegen id : 1]
+(2) ColumnarToRow [codegen id : 37]
Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-(3) Filter [codegen id : 1]
+(3) Filter [codegen id : 37]
Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
Condition : isnotnull(ss_item_sk#1)
-(4) Exchange
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1]
-
-(5) Sort [codegen id : 2]
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0
-
-(6) Scan parquet spark_catalog.default.item
+(4) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(7) ColumnarToRow [codegen id : 19]
+(5) ColumnarToRow [codegen id : 17]
Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
-(8) Filter [codegen id : 19]
+(6) Filter [codegen id : 17]
Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9))
-(9) Scan parquet spark_catalog.default.store_sales
+(7) Scan parquet spark_catalog.default.store_sales
Output [2]: [ss_item_sk#10, ss_sold_date_sk#11]
Batched: true
Location: InMemoryFileIndex []
@@ -145,49 +128,49 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so
PushedFilters: [IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int>
-(10) ColumnarToRow [codegen id : 11]
+(8) ColumnarToRow [codegen id : 9]
Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
-(11) Filter [codegen id : 11]
+(9) Filter [codegen id : 9]
Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
Condition : isnotnull(ss_item_sk#10)
-(12) ReusedExchange [Reuses operator id: 132]
+(10) ReusedExchange [Reuses operator id: 123]
Output [1]: [d_date_sk#13]
-(13) BroadcastHashJoin [codegen id : 11]
+(11) BroadcastHashJoin [codegen id : 9]
Left keys [1]: [ss_sold_date_sk#11]
Right keys [1]: [d_date_sk#13]
Join type: Inner
Join condition: None
-(14) Project [codegen id : 11]
+(12) Project [codegen id : 9]
Output [1]: [ss_item_sk#10]
Input [3]: [ss_item_sk#10, ss_sold_date_sk#11, d_date_sk#13]
-(15) Scan parquet spark_catalog.default.item
+(13) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(16) ColumnarToRow [codegen id : 4]
+(14) ColumnarToRow [codegen id : 2]
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-(17) Filter [codegen id : 4]
+(15) Filter [codegen id : 2]
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
Condition : (((isnotnull(i_item_sk#14) AND isnotnull(i_brand_id#15)) AND isnotnull(i_class_id#16)) AND isnotnull(i_category_id#17))
-(18) Exchange
+(16) Exchange
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=2]
+Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=1]
-(19) Sort [codegen id : 5]
+(17) Sort [codegen id : 3]
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
Arguments: [coalesce(i_brand_id#15, 0) ASC NULLS FIRST, isnull(i_brand_id#15) ASC NULLS FIRST, coalesce(i_class_id#16, 0) ASC NULLS FIRST, isnull(i_class_id#16) ASC NULLS FIRST, coalesce(i_category_id#17, 0) ASC NULLS FIRST, isnull(i_category_id#17) ASC NULLS FIRST], false, 0
-(20) Scan parquet spark_catalog.default.catalog_sales
+(18) Scan parquet spark_catalog.default.catalog_sales
Output [2]: [cs_item_sk#18, cs_sold_date_sk#19]
Batched: true
Location: InMemoryFileIndex []
@@ -195,109 +178,109 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_so
PushedFilters: [IsNotNull(cs_item_sk)]
ReadSchema: struct<cs_item_sk:int>
-(21) ColumnarToRow [codegen id : 8]
+(19) ColumnarToRow [codegen id : 6]
Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
-(22) Filter [codegen id : 8]
+(20) Filter [codegen id : 6]
Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
Condition : isnotnull(cs_item_sk#18)
-(23) ReusedExchange [Reuses operator id: 132]
+(21) ReusedExchange [Reuses operator id: 123]
Output [1]: [d_date_sk#20]
-(24) BroadcastHashJoin [codegen id : 8]
+(22) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [cs_sold_date_sk#19]
Right keys [1]: [d_date_sk#20]
Join type: Inner
Join condition: None
-(25) Project [codegen id : 8]
+(23) Project [codegen id : 6]
Output [1]: [cs_item_sk#18]
Input [3]: [cs_item_sk#18, cs_sold_date_sk#19, d_date_sk#20]
-(26) Scan parquet spark_catalog.default.item
+(24) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(27) ColumnarToRow [codegen id : 7]
+(25) ColumnarToRow [codegen id : 5]
Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-(28) Filter [codegen id : 7]
+(26) Filter [codegen id : 5]
Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
Condition : isnotnull(i_item_sk#21)
-(29) BroadcastExchange
+(27) BroadcastExchange
Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2]
-(30) BroadcastHashJoin [codegen id : 8]
+(28) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [cs_item_sk#18]
Right keys [1]: [i_item_sk#21]
Join type: Inner
Join condition: None
-(31) Project [codegen id : 8]
+(29) Project [codegen id : 6]
Output [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
Input [5]: [cs_item_sk#18, i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-(32) Exchange
+(30) Exchange
Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=4]
+Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=3]
-(33) Sort [codegen id : 9]
+(31) Sort [codegen id : 7]
Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
Arguments: [coalesce(i_brand_id#22, 0) ASC NULLS FIRST, isnull(i_brand_id#22) ASC NULLS FIRST, coalesce(i_class_id#23, 0) ASC NULLS FIRST, isnull(i_class_id#23) ASC NULLS FIRST, coalesce(i_category_id#24, 0) ASC NULLS FIRST, isnull(i_category_id#24) ASC NULLS FIRST], false, 0
-(34) SortMergeJoin [codegen id : 10]
+(32) SortMergeJoin [codegen id : 8]
Left keys [6]: [coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17)]
Right keys [6]: [coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24)]
Join type: LeftSemi
Join condition: None
-(35) BroadcastExchange
+(33) BroadcastExchange
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
-(36) BroadcastHashJoin [codegen id : 11]
+(34) BroadcastHashJoin [codegen id : 9]
Left keys [1]: [ss_item_sk#10]
Right keys [1]: [i_item_sk#14]
Join type: Inner
Join condition: None
-(37) Project [codegen id : 11]
+(35) Project [codegen id : 9]
Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27]
Input [5]: [ss_item_sk#10, i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-(38) HashAggregate [codegen id : 11]
+(36) HashAggregate [codegen id : 9]
Input [3]: [brand_id#25, class_id#26, category_id#27]
Keys [3]: [brand_id#25, class_id#26, category_id#27]
Functions: []
Aggregate Attributes: []
Results [3]: [brand_id#25, class_id#26, category_id#27]
-(39) Exchange
+(37) Exchange
Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=5]
-(40) HashAggregate [codegen id : 12]
+(38) HashAggregate [codegen id : 10]
Input [3]: [brand_id#25, class_id#26, category_id#27]
Keys [3]: [brand_id#25, class_id#26, category_id#27]
Functions: []
Aggregate Attributes: []
Results [3]: [brand_id#25, class_id#26, category_id#27]
-(41) Exchange
+(39) Exchange
Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=7]
+Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=6]
-(42) Sort [codegen id : 13]
+(40) Sort [codegen id : 11]
Input [3]: [brand_id#25, class_id#26, category_id#27]
Arguments: [coalesce(brand_id#25, 0) ASC NULLS FIRST, isnull(brand_id#25) ASC NULLS FIRST, coalesce(class_id#26, 0) ASC NULLS FIRST, isnull(class_id#26) ASC NULLS FIRST, coalesce(category_id#27, 0) ASC NULLS FIRST, isnull(category_id#27) ASC NULLS FIRST], false, 0
-(43) Scan parquet spark_catalog.default.web_sales
+(41) Scan parquet spark_catalog.default.web_sales
Output [2]: [ws_item_sk#28, ws_sold_date_sk#29]
Batched: true
Location: InMemoryFileIndex []
@@ -305,166 +288,150 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so
PushedFilters: [IsNotNull(ws_item_sk)]
ReadSchema: struct<ws_item_sk:int>
-(44) ColumnarToRow [codegen id : 16]
+(42) ColumnarToRow [codegen id : 14]
Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
-(45) Filter [codegen id : 16]
+(43) Filter [codegen id : 14]
Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
Condition : isnotnull(ws_item_sk#28)
-(46) ReusedExchange [Reuses operator id: 132]
+(44) ReusedExchange [Reuses operator id: 123]
Output [1]: [d_date_sk#30]
-(47) BroadcastHashJoin [codegen id : 16]
+(45) BroadcastHashJoin [codegen id : 14]
Left keys [1]: [ws_sold_date_sk#29]
Right keys [1]: [d_date_sk#30]
Join type: Inner
Join condition: None
-(48) Project [codegen id : 16]
+(46) Project [codegen id : 14]
Output [1]: [ws_item_sk#28]
Input [3]: [ws_item_sk#28, ws_sold_date_sk#29, d_date_sk#30]
-(49) ReusedExchange [Reuses operator id: 29]
+(47) ReusedExchange [Reuses operator id: 27]
Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
-(50) BroadcastHashJoin [codegen id : 16]
+(48) BroadcastHashJoin [codegen id : 14]
Left keys [1]: [ws_item_sk#28]
Right keys [1]: [i_item_sk#31]
Join type: Inner
Join condition: None
-(51) Project [codegen id : 16]
+(49) Project [codegen id : 14]
Output [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
Input [5]: [ws_item_sk#28, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
-(52) Exchange
+(50) Exchange
Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
-Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=8]
+Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=7]
-(53) Sort [codegen id : 17]
+(51) Sort [codegen id : 15]
Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
Arguments: [coalesce(i_brand_id#32, 0) ASC NULLS FIRST, isnull(i_brand_id#32) ASC NULLS FIRST, coalesce(i_class_id#33, 0) ASC NULLS FIRST, isnull(i_class_id#33) ASC NULLS FIRST, coalesce(i_category_id#34, 0) ASC NULLS FIRST, isnull(i_category_id#34) ASC NULLS FIRST], false, 0
-(54) SortMergeJoin [codegen id : 18]
+(52) SortMergeJoin [codegen id : 16]
Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)]
Right keys [6]: [coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34)]
Join type: LeftSemi
Join condition: None
-(55) BroadcastExchange
+(53) BroadcastExchange
Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=9]
+Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=8]
-(56) BroadcastHashJoin [codegen id : 19]
+(54) BroadcastHashJoin [codegen id : 17]
Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Right keys [3]: [brand_id#25, class_id#26, category_id#27]
Join type: Inner
Join condition: None
-(57) Project [codegen id : 19]
+(55) Project [codegen id : 17]
Output [1]: [i_item_sk#6 AS ss_item_sk#35]
Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27]
-(58) Exchange
+(56) BroadcastExchange
Input [1]: [ss_item_sk#35]
-Arguments: hashpartitioning(ss_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=10]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9]
-(59) Sort [codegen id : 20]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(60) SortMergeJoin [codegen id : 43]
+(57) BroadcastHashJoin [codegen id : 37]
Left keys [1]: [ss_item_sk#1]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(61) ReusedExchange [Reuses operator id: 123]
+(58) ReusedExchange [Reuses operator id: 114]
Output [1]: [d_date_sk#36]
-(62) BroadcastHashJoin [codegen id : 43]
+(59) BroadcastHashJoin [codegen id : 37]
Left keys [1]: [ss_sold_date_sk#4]
Right keys [1]: [d_date_sk#36]
Join type: Inner
Join condition: None
-(63) Project [codegen id : 43]
+(60) Project [codegen id : 37]
Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3]
Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36]
-(64) Scan parquet spark_catalog.default.item
+(61) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(65) ColumnarToRow [codegen id : 22]
+(62) ColumnarToRow [codegen id : 36]
Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-(66) Filter [codegen id : 22]
+(63) Filter [codegen id : 36]
Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
Condition : (((isnotnull(i_item_sk#37) AND isnotnull(i_brand_id#38)) AND isnotnull(i_class_id#39)) AND isnotnull(i_category_id#40))
-(67) Exchange
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: hashpartitioning(i_item_sk#37, 5), ENSURE_REQUIREMENTS, [plan_id=11]
-
-(68) Sort [codegen id : 23]
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: [i_item_sk#37 ASC NULLS FIRST], false, 0
-
-(69) ReusedExchange [Reuses operator id: 58]
+(64) ReusedExchange [Reuses operator id: 56]
Output [1]: [ss_item_sk#35]
-(70) Sort [codegen id : 41]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(71) SortMergeJoin [codegen id : 42]
+(65) BroadcastHashJoin [codegen id : 36]
Left keys [1]: [i_item_sk#37]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(72) BroadcastExchange
+(66) BroadcastExchange
Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10]
-(73) BroadcastHashJoin [codegen id : 43]
+(67) BroadcastHashJoin [codegen id : 37]
Left keys [1]: [ss_item_sk#1]
Right keys [1]: [i_item_sk#37]
Join type: Inner
Join condition: None
-(74) Project [codegen id : 43]
+(68) Project [codegen id : 37]
Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-(75) HashAggregate [codegen id : 43]
+(69) HashAggregate [codegen id : 37]
Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
-(76) Exchange
+(70) Exchange
Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
-Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=13]
+Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11]
-(77) HashAggregate [codegen id : 88]
+(71) HashAggregate [codegen id : 76]
Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51]
-(78) Filter [codegen id : 88]
+(72) Filter [codegen id : 76]
Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51]
Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
-(79) Scan parquet spark_catalog.default.store_sales
+(73) Scan parquet spark_catalog.default.store_sales
Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
Batched: true
Location: InMemoryFileIndex []
@@ -472,358 +439,346 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so
PushedFilters: [IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)>
-(80) ColumnarToRow [codegen id : 44]
+(74) ColumnarToRow [codegen id : 74]
Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
-(81) Filter [codegen id : 44]
+(75) Filter [codegen id : 74]
Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
Condition : isnotnull(ss_item_sk#54)
-(82) Exchange
-Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
-Arguments: hashpartitioning(ss_item_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=14]
-
-(83) Sort [codegen id : 45]
-Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
-Arguments: [ss_item_sk#54 ASC NULLS FIRST], false, 0
-
-(84) ReusedExchange [Reuses operator id: 58]
+(76) ReusedExchange [Reuses operator id: 56]
Output [1]: [ss_item_sk#35]
-(85) Sort [codegen id : 63]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(86) SortMergeJoin [codegen id : 86]
+(77) BroadcastHashJoin [codegen id : 74]
Left keys [1]: [ss_item_sk#54]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(87) ReusedExchange [Reuses operator id: 137]
+(78) ReusedExchange [Reuses operator id: 128]
Output [1]: [d_date_sk#59]
-(88) BroadcastHashJoin [codegen id : 86]
+(79) BroadcastHashJoin [codegen id : 74]
Left keys [1]: [ss_sold_date_sk#57]
Right keys [1]: [d_date_sk#59]
Join type: Inner
Join condition: None
-(89) Project [codegen id : 86]
+(80) Project [codegen id : 74]
Output [3]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56]
Input [5]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, d_date_sk#59]
-(90) ReusedExchange [Reuses operator id: 72]
+(81) ReusedExchange [Reuses operator id: 66]
Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63]
-(91) BroadcastHashJoin [codegen id : 86]
+(82) BroadcastHashJoin [codegen id : 74]
Left keys [1]: [ss_item_sk#54]
Right keys [1]: [i_item_sk#60]
Join type: Inner
Join condition: None
-(92) Project [codegen id : 86]
+(83) Project [codegen id : 74]
Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63]
Input [7]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63]
-(93) HashAggregate [codegen id : 86]
+(84) HashAggregate [codegen id : 74]
Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63]
Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)]
Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66]
Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
-(94) Exchange
+(85) Exchange
Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
-Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=15]
+Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=12]
-(95) HashAggregate [codegen id : 87]
+(86) HashAggregate [codegen id : 75]
Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)]
Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71]
Results [6]: [store AS channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74]
-(96) Filter [codegen id : 87]
+(87) Filter [codegen id : 75]
Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
-(97) BroadcastExchange
+(88) BroadcastExchange
Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
-Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=16]
+Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=13]
-(98) BroadcastHashJoin [codegen id : 88]
+(89) BroadcastHashJoin [codegen id : 76]
Left keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
Join type: Inner
Join condition: None
-(99) TakeOrderedAndProject
+(90) TakeOrderedAndProject
Input [12]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
Arguments: 100, [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, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
===== Subqueries =====
-Subquery:1 Hosting operator id = 78 Hosting Expression = Subquery scalar-subquery#52, [id=#53]
-* HashAggregate (118)
-+- Exchange (117)
- +- * HashAggregate (116)
- +- Union (115)
- :- * Project (104)
- : +- * BroadcastHashJoin Inner BuildRight (103)
- : :- * ColumnarToRow (101)
- : : +- Scan parquet spark_catalog.default.store_sales (100)
- : +- ReusedExchange (102)
- :- * Project (109)
- : +- * BroadcastHashJoin Inner BuildRight (108)
- : :- * ColumnarToRow (106)
- : : +- Scan parquet spark_catalog.default.catalog_sales (105)
- : +- ReusedExchange (107)
- +- * Project (114)
- +- * BroadcastHashJoin Inner BuildRight (113)
- :- * ColumnarToRow (111)
- : +- Scan parquet spark_catalog.default.web_sales (110)
- +- ReusedExchange (112)
-
-
-(100) Scan parquet spark_catalog.default.store_sales
+Subquery:1 Hosting operator id = 72 Hosting Expression = Subquery scalar-subquery#52, [id=#53]
+* HashAggregate (109)
++- Exchange (108)
+ +- * HashAggregate (107)
+ +- Union (106)
+ :- * Project (95)
+ : +- * BroadcastHashJoin Inner BuildRight (94)
+ : :- * ColumnarToRow (92)
+ : : +- Scan parquet spark_catalog.default.store_sales (91)
+ : +- ReusedExchange (93)
+ :- * Project (100)
+ : +- * BroadcastHashJoin Inner BuildRight (99)
+ : :- * ColumnarToRow (97)
+ : : +- Scan parquet spark_catalog.default.catalog_sales (96)
+ : +- ReusedExchange (98)
+ +- * Project (105)
+ +- * BroadcastHashJoin Inner BuildRight (104)
+ :- * ColumnarToRow (102)
+ : +- Scan parquet spark_catalog.default.web_sales (101)
+ +- ReusedExchange (103)
+
+
+(91) Scan parquet spark_catalog.default.store_sales
Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)]
ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)>
-(101) ColumnarToRow [codegen id : 2]
+(92) ColumnarToRow [codegen id : 2]
Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77]
-(102) ReusedExchange [Reuses operator id: 132]
+(93) ReusedExchange [Reuses operator id: 123]
Output [1]: [d_date_sk#78]
-(103) BroadcastHashJoin [codegen id : 2]
+(94) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [ss_sold_date_sk#77]
Right keys [1]: [d_date_sk#78]
Join type: Inner
Join condition: None
-(104) Project [codegen id : 2]
+(95) Project [codegen id : 2]
Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80]
Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78]
-(105) Scan parquet spark_catalog.default.catalog_sales
+(96) Scan parquet spark_catalog.default.catalog_sales
Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)]
ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)>
-(106) ColumnarToRow [codegen id : 4]
+(97) ColumnarToRow [codegen id : 4]
Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83]
-(107) ReusedExchange [Reuses operator id: 132]
+(98) ReusedExchange [Reuses operator id: 123]
Output [1]: [d_date_sk#84]
-(108) BroadcastHashJoin [codegen id : 4]
+(99) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [cs_sold_date_sk#83]
Right keys [1]: [d_date_sk#84]
Join type: Inner
Join condition: None
-(109) Project [codegen id : 4]
+(100) Project [codegen id : 4]
Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86]
Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84]
-(110) Scan parquet spark_catalog.default.web_sales
+(101) Scan parquet spark_catalog.default.web_sales
Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)]
ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)>
-(111) ColumnarToRow [codegen id : 6]
+(102) ColumnarToRow [codegen id : 6]
Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89]
-(112) ReusedExchange [Reuses operator id: 132]
+(103) ReusedExchange [Reuses operator id: 123]
Output [1]: [d_date_sk#90]
-(113) BroadcastHashJoin [codegen id : 6]
+(104) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [ws_sold_date_sk#89]
Right keys [1]: [d_date_sk#90]
Join type: Inner
Join condition: None
-(114) Project [codegen id : 6]
+(105) Project [codegen id : 6]
Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92]
Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90]
-(115) Union
+(106) Union
-(116) HashAggregate [codegen id : 7]
+(107) HashAggregate [codegen id : 7]
Input [2]: [quantity#79, list_price#80]
Keys: []
Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
Aggregate Attributes [2]: [sum#93, count#94]
Results [2]: [sum#95, count#96]
-(117) Exchange
+(108) Exchange
Input [2]: [sum#95, count#96]
-Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14]
-(118) HashAggregate [codegen id : 8]
+(109) HashAggregate [codegen id : 8]
Input [2]: [sum#95, count#96]
Keys: []
Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97]
Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98]
-Subquery:2 Hosting operator id = 100 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12
+Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12
-Subquery:3 Hosting operator id = 105 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12
+Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12
-Subquery:4 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12
+Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12
Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (123)
-+- * Project (122)
- +- * Filter (121)
- +- * ColumnarToRow (120)
- +- Scan parquet spark_catalog.default.date_dim (119)
+BroadcastExchange (114)
++- * Project (113)
+ +- * Filter (112)
+ +- * ColumnarToRow (111)
+ +- Scan parquet spark_catalog.default.date_dim (110)
-(119) Scan parquet spark_catalog.default.date_dim
+(110) Scan parquet spark_catalog.default.date_dim
Output [2]: [d_date_sk#36, d_week_seq#99]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_week_seq:int>
-(120) ColumnarToRow [codegen id : 1]
+(111) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#36, d_week_seq#99]
-(121) Filter [codegen id : 1]
+(112) Filter [codegen id : 1]
Input [2]: [d_date_sk#36, d_week_seq#99]
Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#36))
-(122) Project [codegen id : 1]
+(113) Project [codegen id : 1]
Output [1]: [d_date_sk#36]
Input [2]: [d_date_sk#36, d_week_seq#99]
-(123) BroadcastExchange
+(114) BroadcastExchange
Input [1]: [d_date_sk#36]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=18]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15]
-Subquery:6 Hosting operator id = 121 Hosting Expression = Subquery scalar-subquery#100, [id=#101]
-* Project (127)
-+- * Filter (126)
- +- * ColumnarToRow (125)
- +- Scan parquet spark_catalog.default.date_dim (124)
+Subquery:6 Hosting operator id = 112 Hosting Expression = Subquery scalar-subquery#100, [id=#101]
+* Project (118)
++- * Filter (117)
+ +- * ColumnarToRow (116)
+ +- Scan parquet spark_catalog.default.date_dim (115)
-(124) Scan parquet spark_catalog.default.date_dim
+(115) Scan parquet spark_catalog.default.date_dim
Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)]
ReadSchema: struct<d_week_seq:int,d_year:int,d_moy:int,d_dom:int>
-(125) ColumnarToRow [codegen id : 1]
+(116) ColumnarToRow [codegen id : 1]
Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
-(126) Filter [codegen id : 1]
+(117) Filter [codegen id : 1]
Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 1999)) AND (d_moy#104 = 12)) AND (d_dom#105 = 16))
-(127) Project [codegen id : 1]
+(118) Project [codegen id : 1]
Output [1]: [d_week_seq#102]
Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
-Subquery:7 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
-BroadcastExchange (132)
-+- * Project (131)
- +- * Filter (130)
- +- * ColumnarToRow (129)
- +- Scan parquet spark_catalog.default.date_dim (128)
+Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
+BroadcastExchange (123)
++- * Project (122)
+ +- * Filter (121)
+ +- * ColumnarToRow (120)
+ +- Scan parquet spark_catalog.default.date_dim (119)
-(128) Scan parquet spark_catalog.default.date_dim
+(119) Scan parquet spark_catalog.default.date_dim
Output [2]: [d_date_sk#13, d_year#106]
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<d_date_sk:int,d_year:int>
-(129) ColumnarToRow [codegen id : 1]
+(120) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#13, d_year#106]
-(130) Filter [codegen id : 1]
+(121) Filter [codegen id : 1]
Input [2]: [d_date_sk#13, d_year#106]
Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1998)) AND (d_year#106 <= 2000)) AND isnotnull(d_date_sk#13))
-(131) Project [codegen id : 1]
+(122) Project [codegen id : 1]
Output [1]: [d_date_sk#13]
Input [2]: [d_date_sk#13, d_year#106]
-(132) BroadcastExchange
+(123) BroadcastExchange
Input [1]: [d_date_sk#13]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16]
-Subquery:8 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
+Subquery:8 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
-Subquery:9 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
+Subquery:9 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
-Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
+Subquery:10 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
-Subquery:11 Hosting operator id = 79 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58
-BroadcastExchange (137)
-+- * Project (136)
- +- * Filter (135)
- +- * ColumnarToRow (134)
- +- Scan parquet spark_catalog.default.date_dim (133)
+Subquery:11 Hosting operator id = 73 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58
+BroadcastExchange (128)
++- * Project (127)
+ +- * Filter (126)
+ +- * ColumnarToRow (125)
+ +- Scan parquet spark_catalog.default.date_dim (124)
-(133) Scan parquet spark_catalog.default.date_dim
+(124) Scan parquet spark_catalog.default.date_dim
Output [2]: [d_date_sk#59, d_week_seq#107]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_week_seq:int>
-(134) ColumnarToRow [codegen id : 1]
+(125) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#59, d_week_seq#107]
-(135) Filter [codegen id : 1]
+(126) Filter [codegen id : 1]
Input [2]: [d_date_sk#59, d_week_seq#107]
Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#59))
-(136) Project [codegen id : 1]
+(127) Project [codegen id : 1]
Output [1]: [d_date_sk#59]
Input [2]: [d_date_sk#59, d_week_seq#107]
-(137) BroadcastExchange
+(128) BroadcastExchange
Input [1]: [d_date_sk#59]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17]
-Subquery:12 Hosting operator id = 135 Hosting Expression = Subquery scalar-subquery#108, [id=#109]
-* Project (141)
-+- * Filter (140)
- +- * ColumnarToRow (139)
- +- Scan parquet spark_catalog.default.date_dim (138)
+Subquery:12 Hosting operator id = 126 Hosting Expression = Subquery scalar-subquery#108, [id=#109]
+* Project (132)
++- * Filter (131)
+ +- * ColumnarToRow (130)
+ +- Scan parquet spark_catalog.default.date_dim (129)
-(138) Scan parquet spark_catalog.default.date_dim
+(129) Scan parquet spark_catalog.default.date_dim
Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)]
ReadSchema: struct<d_week_seq:int,d_year:int,d_moy:int,d_dom:int>
-(139) ColumnarToRow [codegen id : 1]
+(130) ColumnarToRow [codegen id : 1]
Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
-(140) Filter [codegen id : 1]
+(131) Filter [codegen id : 1]
Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1998)) AND (d_moy#112 = 12)) AND (d_dom#113 = 16))
-(141) Project [codegen id : 1]
+(132) Project [codegen id : 1]
Output [1]: [d_week_seq#110]
Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt
index 8c67fff5b14..edd34864986 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt
@@ -1,12 +1,12 @@
TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
- WholeStageCodegen (88)
+ WholeStageCodegen (76)
BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
Filter [sales]
Subquery #4
WholeStageCodegen (8)
HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
InputAdapter
- Exchange #16
+ Exchange #14
WholeStageCodegen (7)
HashAggregate [quantity,list_price] [sum,count,sum,count]
InputAdapter
@@ -19,7 +19,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #3
InputAdapter
- ReusedExchange [d_date_sk] #8
+ ReusedExchange [d_date_sk] #7
WholeStageCodegen (4)
Project [cs_quantity,cs_list_price]
BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
@@ -28,7 +28,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #3
InputAdapter
- ReusedExchange [d_date_sk] #8
+ ReusedExchange [d_date_sk] #7
WholeStageCodegen (6)
Project [ws_quantity,ws_list_price]
BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
@@ -37,211 +37,184 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #3
InputAdapter
- ReusedExchange [d_date_sk] #8
+ ReusedExchange [d_date_sk] #7
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] #1
- WholeStageCodegen (43)
+ 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]
- SortMergeJoin [ss_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (2)
- Sort [ss_item_sk]
- InputAdapter
- Exchange [ss_item_sk] #2
- WholeStageCodegen (1)
- 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_week_seq,d_date_sk]
- Subquery #2
- WholeStageCodegen (1)
- Project [d_week_seq]
- Filter [d_year,d_moy,d_dom]
- ColumnarToRow
- InputAdapter
- Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
- ColumnarToRow
- InputAdapter
- Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
- InputAdapter
- WholeStageCodegen (20)
- Sort [ss_item_sk]
- InputAdapter
- Exchange [ss_item_sk] #4
- WholeStageCodegen (19)
- 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]
+ 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 #2
+ WholeStageCodegen (1)
+ Project [d_date_sk]
+ Filter [d_week_seq,d_date_sk]
+ Subquery #2
+ WholeStageCodegen (1)
+ Project [d_week_seq]
+ Filter [d_year,d_moy,d_dom]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
ColumnarToRow
InputAdapter
- Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
- InputAdapter
- BroadcastExchange #5
- WholeStageCodegen (18)
- 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_week_seq]
+ InputAdapter
+ BroadcastExchange #3
+ 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 #4
+ 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 (13)
- Sort [brand_id,class_id,category_id]
- InputAdapter
- Exchange [brand_id,class_id,category_id] #6
- WholeStageCodegen (12)
- HashAggregate [brand_id,class_id,category_id]
- InputAdapter
- Exchange [brand_id,class_id,category_id] #7
- WholeStageCodegen (11)
- 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] #3
- 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 (10)
- 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] #5
+ WholeStageCodegen (10)
+ HashAggregate [brand_id,class_id,category_id]
+ InputAdapter
+ Exchange [brand_id,class_id,category_id] #6
+ 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] #3
+ 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
+ ReusedExchange [d_date_sk] #7
+ InputAdapter
+ BroadcastExchange #8
+ 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 (5)
- Sort [i_brand_id,i_class_id,i_category_id]
- InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #10
- WholeStageCodegen (4)
- 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] #9
+ 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 (9)
- Sort [i_brand_id,i_class_id,i_category_id]
- InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #11
- WholeStageCodegen (8)
- 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] #3
+ Exchange [i_brand_id,i_class_id,i_category_id] #10
+ 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] #3
+ InputAdapter
+ ReusedExchange [d_date_sk] #7
+ InputAdapter
+ BroadcastExchange #11
+ WholeStageCodegen (5)
+ Filter [i_item_sk]
+ ColumnarToRow
InputAdapter
- ReusedExchange [d_date_sk] #8
- InputAdapter
- BroadcastExchange #12
- WholeStageCodegen (7)
- 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 (17)
- Sort [i_brand_id,i_class_id,i_category_id]
- InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #13
- WholeStageCodegen (16)
- 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] #3
- InputAdapter
- ReusedExchange [d_date_sk] #8
- InputAdapter
- ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12
+ Exchange [i_brand_id,i_class_id,i_category_id] #12
+ 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] #3
+ InputAdapter
+ ReusedExchange [d_date_sk] #7
+ InputAdapter
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11
InputAdapter
- ReusedExchange [d_date_sk] #3
+ ReusedExchange [d_date_sk] #2
InputAdapter
- BroadcastExchange #14
- WholeStageCodegen (42)
- SortMergeJoin [i_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (23)
- Sort [i_item_sk]
- InputAdapter
- Exchange [i_item_sk] #15
- WholeStageCodegen (22)
- 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]
+ BroadcastExchange #13
+ WholeStageCodegen (36)
+ BroadcastHashJoin [i_item_sk,ss_item_sk]
+ 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 (41)
- Sort [ss_item_sk]
- InputAdapter
- ReusedExchange [ss_item_sk] #4
+ ReusedExchange [ss_item_sk] #3
InputAdapter
- BroadcastExchange #17
- WholeStageCodegen (87)
+ BroadcastExchange #15
+ WholeStageCodegen (75)
Filter [sales]
ReusedSubquery [average_sales] #4
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] #18
- WholeStageCodegen (86)
+ Exchange [i_brand_id,i_class_id,i_category_id] #16
+ WholeStageCodegen (74)
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]
- SortMergeJoin [ss_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (45)
- Sort [ss_item_sk]
- InputAdapter
- Exchange [ss_item_sk] #19
- WholeStageCodegen (44)
- 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] #5
- BroadcastExchange #20
- WholeStageCodegen (1)
- Project [d_date_sk]
- Filter [d_week_seq,d_date_sk]
- Subquery #6
- WholeStageCodegen (1)
- Project [d_week_seq]
- Filter [d_year,d_moy,d_dom]
- ColumnarToRow
- InputAdapter
- Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
- ColumnarToRow
- InputAdapter
- Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
+ 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] #5
+ BroadcastExchange #17
+ WholeStageCodegen (1)
+ Project [d_date_sk]
+ Filter [d_week_seq,d_date_sk]
+ Subquery #6
+ WholeStageCodegen (1)
+ Project [d_week_seq]
+ Filter [d_year,d_moy,d_dom]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
InputAdapter
- WholeStageCodegen (63)
- Sort [ss_item_sk]
- InputAdapter
- ReusedExchange [ss_item_sk] #4
+ ReusedExchange [ss_item_sk] #3
InputAdapter
- ReusedExchange [d_date_sk] #20
+ ReusedExchange [d_date_sk] #17
InputAdapter
- ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13
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 f3db50d8f15..9527bf6a061 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,147 +1,135 @@
== Physical Plan ==
-TakeOrderedAndProject (143)
-+- * HashAggregate (142)
- +- Exchange (141)
- +- * HashAggregate (140)
- +- Union (139)
- :- * HashAggregate (118)
- : +- Exchange (117)
- : +- * HashAggregate (116)
- : +- Union (115)
- : :- * Filter (78)
- : : +- * HashAggregate (77)
- : : +- Exchange (76)
- : : +- * HashAggregate (75)
- : : +- * Project (74)
- : : +- * BroadcastHashJoin Inner BuildRight (73)
- : : :- * Project (63)
- : : : +- * BroadcastHashJoin Inner BuildRight (62)
- : : : :- * SortMergeJoin LeftSemi (60)
- : : : : :- * Sort (5)
- : : : : : +- Exchange (4)
- : : : : : +- * Filter (3)
- : : : : : +- * ColumnarToRow (2)
- : : : : : +- Scan parquet spark_catalog.default.store_sales (1)
- : : : : +- * Sort (59)
- : : : : +- Exchange (58)
- : : : : +- * Project (57)
- : : : : +- * BroadcastHashJoin Inner BuildRight (56)
- : : : : :- * Filter (8)
- : : : : : +- * ColumnarToRow (7)
- : : : : : +- Scan parquet spark_catalog.default.item (6)
- : : : : +- BroadcastExchange (55)
- : : : : +- * SortMergeJoin LeftSemi (54)
- : : : : :- * Sort (42)
- : : : : : +- Exchange (41)
- : : : : : +- * HashAggregate (40)
- : : : : : +- Exchange (39)
- : : : : : +- * HashAggregate (38)
- : : : : : +- * Project (37)
- : : : : : +- * BroadcastHashJoin Inner BuildRight (36)
- : : : : : :- * Project (14)
- : : : : : : +- * BroadcastHashJoin Inner BuildRight (13)
- : : : : : : :- * Filter (11)
- : : : : : : : +- * ColumnarToRow (10)
- : : : : : : : +- Scan parquet spark_catalog.default.store_sales (9)
- : : : : : : +- ReusedExchange (12)
- : : : : : +- BroadcastExchange (35)
- : : : : : +- * SortMergeJoin LeftSemi (34)
- : : : : : :- * Sort (19)
- : : : : : : +- Exchange (18)
- : : : : : : +- * Filter (17)
- : : : : : : +- * ColumnarToRow (16)
- : : : : : : +- Scan parquet spark_catalog.default.item (15)
- : : : : : +- * Sort (33)
- : : : : : +- Exchange (32)
- : : : : : +- * Project (31)
- : : : : : +- * BroadcastHashJoin Inner BuildRight (30)
- : : : : : :- * Project (25)
- : : : : : : +- * BroadcastHashJoin Inner BuildRight (24)
- : : : : : : :- * Filter (22)
- : : : : : : : +- * ColumnarToRow (21)
- : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (20)
- : : : : : : +- ReusedExchange (23)
- : : : : : +- BroadcastExchange (29)
- : : : : : +- * Filter (28)
- : : : : : +- * ColumnarToRow (27)
- : : : : : +- Scan parquet spark_catalog.default.item (26)
- : : : : +- * Sort (53)
- : : : : +- Exchange (52)
- : : : : +- * Project (51)
- : : : : +- * BroadcastHashJoin Inner BuildRight (50)
- : : : : :- * Project (48)
- : : : : : +- * BroadcastHashJoin Inner BuildRight (47)
- : : : : : :- * Filter (45)
- : : : : : : +- * ColumnarToRow (44)
- : : : : : : +- Scan parquet spark_catalog.default.web_sales (43)
- : : : : : +- ReusedExchange (46)
- : : : : +- ReusedExchange (49)
- : : : +- ReusedExchange (61)
- : : +- BroadcastExchange (72)
- : : +- * SortMergeJoin LeftSemi (71)
- : : :- * Sort (68)
- : : : +- Exchange (67)
- : : : +- * Filter (66)
- : : : +- * ColumnarToRow (65)
- : : : +- Scan parquet spark_catalog.default.item (64)
- : : +- * Sort (70)
- : : +- ReusedExchange (69)
- : :- * Filter (96)
- : : +- * HashAggregate (95)
- : : +- Exchange (94)
- : : +- * HashAggregate (93)
- : : +- * Project (92)
- : : +- * BroadcastHashJoin Inner BuildRight (91)
- : : :- * Project (89)
- : : : +- * BroadcastHashJoin Inner BuildRight (88)
- : : : :- * SortMergeJoin LeftSemi (86)
- : : : : :- * Sort (83)
- : : : : : +- Exchange (82)
- : : : : : +- * Filter (81)
- : : : : : +- * ColumnarToRow (80)
- : : : : : +- Scan parquet spark_catalog.default.catalog_sales (79)
- : : : : +- * Sort (85)
- : : : : +- ReusedExchange (84)
- : : : +- ReusedExchange (87)
- : : +- ReusedExchange (90)
- : +- * Filter (114)
- : +- * HashAggregate (113)
- : +- Exchange (112)
- : +- * HashAggregate (111)
- : +- * Project (110)
- : +- * BroadcastHashJoin Inner BuildRight (109)
- : :- * Project (107)
- : : +- * BroadcastHashJoin Inner BuildRight (106)
- : : :- * SortMergeJoin LeftSemi (104)
- : : : :- * Sort (101)
- : : : : +- Exchange (100)
- : : : : +- * Filter (99)
- : : : : +- * ColumnarToRow (98)
- : : : : +- Scan parquet spark_catalog.default.web_sales (97)
- : : : +- * Sort (103)
- : : : +- ReusedExchange (102)
- : : +- ReusedExchange (105)
- : +- ReusedExchange (108)
- :- * HashAggregate (123)
- : +- Exchange (122)
- : +- * HashAggregate (121)
- : +- * HashAggregate (120)
- : +- ReusedExchange (119)
- :- * HashAggregate (128)
- : +- Exchange (127)
- : +- * HashAggregate (126)
- : +- * HashAggregate (125)
- : +- ReusedExchange (124)
- :- * HashAggregate (133)
- : +- Exchange (132)
- : +- * HashAggregate (131)
- : +- * HashAggregate (130)
- : +- ReusedExchange (129)
- +- * HashAggregate (138)
- +- Exchange (137)
- +- * HashAggregate (136)
- +- * HashAggregate (135)
- +- ReusedExchange (134)
+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)
(1) Scan parquet spark_catalog.default.store_sales
@@ -152,36 +140,28 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol
PushedFilters: [IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)>
-(2) ColumnarToRow [codegen id : 1]
+(2) ColumnarToRow [codegen id : 37]
Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-(3) Filter [codegen id : 1]
+(3) Filter [codegen id : 37]
Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
Condition : isnotnull(ss_item_sk#1)
-(4) Exchange
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1]
-
-(5) Sort [codegen id : 2]
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0
-
-(6) Scan parquet spark_catalog.default.item
+(4) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(7) ColumnarToRow [codegen id : 19]
+(5) ColumnarToRow [codegen id : 17]
Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
-(8) Filter [codegen id : 19]
+(6) Filter [codegen id : 17]
Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9))
-(9) Scan parquet spark_catalog.default.store_sales
+(7) Scan parquet spark_catalog.default.store_sales
Output [2]: [ss_item_sk#10, ss_sold_date_sk#11]
Batched: true
Location: InMemoryFileIndex []
@@ -189,49 +169,49 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so
PushedFilters: [IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int>
-(10) ColumnarToRow [codegen id : 11]
+(8) ColumnarToRow [codegen id : 9]
Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
-(11) Filter [codegen id : 11]
+(9) Filter [codegen id : 9]
Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
Condition : isnotnull(ss_item_sk#10)
-(12) ReusedExchange [Reuses operator id: 177]
+(10) ReusedExchange [Reuses operator id: 165]
Output [1]: [d_date_sk#13]
-(13) BroadcastHashJoin [codegen id : 11]
+(11) BroadcastHashJoin [codegen id : 9]
Left keys [1]: [ss_sold_date_sk#11]
Right keys [1]: [d_date_sk#13]
Join type: Inner
Join condition: None
-(14) Project [codegen id : 11]
+(12) Project [codegen id : 9]
Output [1]: [ss_item_sk#10]
Input [3]: [ss_item_sk#10, ss_sold_date_sk#11, d_date_sk#13]
-(15) Scan parquet spark_catalog.default.item
+(13) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(16) ColumnarToRow [codegen id : 4]
+(14) ColumnarToRow [codegen id : 2]
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-(17) Filter [codegen id : 4]
+(15) Filter [codegen id : 2]
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
Condition : (((isnotnull(i_item_sk#14) AND isnotnull(i_brand_id#15)) AND isnotnull(i_class_id#16)) AND isnotnull(i_category_id#17))
-(18) Exchange
+(16) Exchange
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=2]
+Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=1]
-(19) Sort [codegen id : 5]
+(17) Sort [codegen id : 3]
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
Arguments: [coalesce(i_brand_id#15, 0) ASC NULLS FIRST, isnull(i_brand_id#15) ASC NULLS FIRST, coalesce(i_class_id#16, 0) ASC NULLS FIRST, isnull(i_class_id#16) ASC NULLS FIRST, coalesce(i_category_id#17, 0) ASC NULLS FIRST, isnull(i_category_id#17) ASC NULLS FIRST], false, 0
-(20) Scan parquet spark_catalog.default.catalog_sales
+(18) Scan parquet spark_catalog.default.catalog_sales
Output [2]: [cs_item_sk#18, cs_sold_date_sk#19]
Batched: true
Location: InMemoryFileIndex []
@@ -239,109 +219,109 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_so
PushedFilters: [IsNotNull(cs_item_sk)]
ReadSchema: struct<cs_item_sk:int>
-(21) ColumnarToRow [codegen id : 8]
+(19) ColumnarToRow [codegen id : 6]
Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
-(22) Filter [codegen id : 8]
+(20) Filter [codegen id : 6]
Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
Condition : isnotnull(cs_item_sk#18)
-(23) ReusedExchange [Reuses operator id: 177]
+(21) ReusedExchange [Reuses operator id: 165]
Output [1]: [d_date_sk#20]
-(24) BroadcastHashJoin [codegen id : 8]
+(22) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [cs_sold_date_sk#19]
Right keys [1]: [d_date_sk#20]
Join type: Inner
Join condition: None
-(25) Project [codegen id : 8]
+(23) Project [codegen id : 6]
Output [1]: [cs_item_sk#18]
Input [3]: [cs_item_sk#18, cs_sold_date_sk#19, d_date_sk#20]
-(26) Scan parquet spark_catalog.default.item
+(24) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(27) ColumnarToRow [codegen id : 7]
+(25) ColumnarToRow [codegen id : 5]
Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-(28) Filter [codegen id : 7]
+(26) Filter [codegen id : 5]
Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
Condition : isnotnull(i_item_sk#21)
-(29) BroadcastExchange
+(27) BroadcastExchange
Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2]
-(30) BroadcastHashJoin [codegen id : 8]
+(28) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [cs_item_sk#18]
Right keys [1]: [i_item_sk#21]
Join type: Inner
Join condition: None
-(31) Project [codegen id : 8]
+(29) Project [codegen id : 6]
Output [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
Input [5]: [cs_item_sk#18, i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-(32) Exchange
+(30) Exchange
Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=4]
+Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=3]
-(33) Sort [codegen id : 9]
+(31) Sort [codegen id : 7]
Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
Arguments: [coalesce(i_brand_id#22, 0) ASC NULLS FIRST, isnull(i_brand_id#22) ASC NULLS FIRST, coalesce(i_class_id#23, 0) ASC NULLS FIRST, isnull(i_class_id#23) ASC NULLS FIRST, coalesce(i_category_id#24, 0) ASC NULLS FIRST, isnull(i_category_id#24) ASC NULLS FIRST], false, 0
-(34) SortMergeJoin [codegen id : 10]
+(32) SortMergeJoin [codegen id : 8]
Left keys [6]: [coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17)]
Right keys [6]: [coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24)]
Join type: LeftSemi
Join condition: None
-(35) BroadcastExchange
+(33) BroadcastExchange
Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
-(36) BroadcastHashJoin [codegen id : 11]
+(34) BroadcastHashJoin [codegen id : 9]
Left keys [1]: [ss_item_sk#10]
Right keys [1]: [i_item_sk#14]
Join type: Inner
Join condition: None
-(37) Project [codegen id : 11]
+(35) Project [codegen id : 9]
Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27]
Input [5]: [ss_item_sk#10, i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-(38) HashAggregate [codegen id : 11]
+(36) HashAggregate [codegen id : 9]
Input [3]: [brand_id#25, class_id#26, category_id#27]
Keys [3]: [brand_id#25, class_id#26, category_id#27]
Functions: []
Aggregate Attributes: []
Results [3]: [brand_id#25, class_id#26, category_id#27]
-(39) Exchange
+(37) Exchange
Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=5]
-(40) HashAggregate [codegen id : 12]
+(38) HashAggregate [codegen id : 10]
Input [3]: [brand_id#25, class_id#26, category_id#27]
Keys [3]: [brand_id#25, class_id#26, category_id#27]
Functions: []
Aggregate Attributes: []
Results [3]: [brand_id#25, class_id#26, category_id#27]
-(41) Exchange
+(39) Exchange
Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=7]
+Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=6]
-(42) Sort [codegen id : 13]
+(40) Sort [codegen id : 11]
Input [3]: [brand_id#25, class_id#26, category_id#27]
Arguments: [coalesce(brand_id#25, 0) ASC NULLS FIRST, isnull(brand_id#25) ASC NULLS FIRST, coalesce(class_id#26, 0) ASC NULLS FIRST, isnull(class_id#26) ASC NULLS FIRST, coalesce(category_id#27, 0) ASC NULLS FIRST, isnull(category_id#27) ASC NULLS FIRST], false, 0
-(43) Scan parquet spark_catalog.default.web_sales
+(41) Scan parquet spark_catalog.default.web_sales
Output [2]: [ws_item_sk#28, ws_sold_date_sk#29]
Batched: true
Location: InMemoryFileIndex []
@@ -349,166 +329,150 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so
PushedFilters: [IsNotNull(ws_item_sk)]
ReadSchema: struct<ws_item_sk:int>
-(44) ColumnarToRow [codegen id : 16]
+(42) ColumnarToRow [codegen id : 14]
Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
-(45) Filter [codegen id : 16]
+(43) Filter [codegen id : 14]
Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
Condition : isnotnull(ws_item_sk#28)
-(46) ReusedExchange [Reuses operator id: 177]
+(44) ReusedExchange [Reuses operator id: 165]
Output [1]: [d_date_sk#30]
-(47) BroadcastHashJoin [codegen id : 16]
+(45) BroadcastHashJoin [codegen id : 14]
Left keys [1]: [ws_sold_date_sk#29]
Right keys [1]: [d_date_sk#30]
Join type: Inner
Join condition: None
-(48) Project [codegen id : 16]
+(46) Project [codegen id : 14]
Output [1]: [ws_item_sk#28]
Input [3]: [ws_item_sk#28, ws_sold_date_sk#29, d_date_sk#30]
-(49) ReusedExchange [Reuses operator id: 29]
+(47) ReusedExchange [Reuses operator id: 27]
Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
-(50) BroadcastHashJoin [codegen id : 16]
+(48) BroadcastHashJoin [codegen id : 14]
Left keys [1]: [ws_item_sk#28]
Right keys [1]: [i_item_sk#31]
Join type: Inner
Join condition: None
-(51) Project [codegen id : 16]
+(49) Project [codegen id : 14]
Output [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
Input [5]: [ws_item_sk#28, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
-(52) Exchange
+(50) Exchange
Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
-Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=8]
+Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=7]
-(53) Sort [codegen id : 17]
+(51) Sort [codegen id : 15]
Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
Arguments: [coalesce(i_brand_id#32, 0) ASC NULLS FIRST, isnull(i_brand_id#32) ASC NULLS FIRST, coalesce(i_class_id#33, 0) ASC NULLS FIRST, isnull(i_class_id#33) ASC NULLS FIRST, coalesce(i_category_id#34, 0) ASC NULLS FIRST, isnull(i_category_id#34) ASC NULLS FIRST], false, 0
-(54) SortMergeJoin [codegen id : 18]
+(52) SortMergeJoin [codegen id : 16]
Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)]
Right keys [6]: [coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34)]
Join type: LeftSemi
Join condition: None
-(55) BroadcastExchange
+(53) BroadcastExchange
Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=9]
+Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=8]
-(56) BroadcastHashJoin [codegen id : 19]
+(54) BroadcastHashJoin [codegen id : 17]
Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Right keys [3]: [brand_id#25, class_id#26, category_id#27]
Join type: Inner
Join condition: None
-(57) Project [codegen id : 19]
+(55) Project [codegen id : 17]
Output [1]: [i_item_sk#6 AS ss_item_sk#35]
Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27]
-(58) Exchange
-Input [1]: [ss_item_sk#35]
-Arguments: hashpartitioning(ss_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=10]
-
-(59) Sort [codegen id : 20]
+(56) BroadcastExchange
Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9]
-(60) SortMergeJoin [codegen id : 43]
+(57) BroadcastHashJoin [codegen id : 37]
Left keys [1]: [ss_item_sk#1]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(61) ReusedExchange [Reuses operator id: 172]
+(58) ReusedExchange [Reuses operator id: 160]
Output [1]: [d_date_sk#36]
-(62) BroadcastHashJoin [codegen id : 43]
+(59) BroadcastHashJoin [codegen id : 37]
Left keys [1]: [ss_sold_date_sk#4]
Right keys [1]: [d_date_sk#36]
Join type: Inner
Join condition: None
-(63) Project [codegen id : 43]
+(60) Project [codegen id : 37]
Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3]
Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36]
-(64) Scan parquet spark_catalog.default.item
+(61) Scan parquet spark_catalog.default.item
Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk)]
ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
-(65) ColumnarToRow [codegen id : 22]
+(62) ColumnarToRow [codegen id : 36]
Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-(66) Filter [codegen id : 22]
+(63) Filter [codegen id : 36]
Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
Condition : isnotnull(i_item_sk#37)
-(67) Exchange
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: hashpartitioning(i_item_sk#37, 5), ENSURE_REQUIREMENTS, [plan_id=11]
-
-(68) Sort [codegen id : 23]
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: [i_item_sk#37 ASC NULLS FIRST], false, 0
-
-(69) ReusedExchange [Reuses operator id: 58]
+(64) ReusedExchange [Reuses operator id: 56]
Output [1]: [ss_item_sk#35]
-(70) Sort [codegen id : 41]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(71) SortMergeJoin [codegen id : 42]
+(65) BroadcastHashJoin [codegen id : 36]
Left keys [1]: [i_item_sk#37]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(72) BroadcastExchange
+(66) BroadcastExchange
Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10]
-(73) BroadcastHashJoin [codegen id : 43]
+(67) BroadcastHashJoin [codegen id : 37]
Left keys [1]: [ss_item_sk#1]
Right keys [1]: [i_item_sk#37]
Join type: Inner
Join condition: None
-(74) Project [codegen id : 43]
+(68) Project [codegen id : 37]
Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-(75) HashAggregate [codegen id : 43]
+(69) HashAggregate [codegen id : 37]
Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
-(76) Exchange
+(70) Exchange
Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
-Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=13]
+Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11]
-(77) HashAggregate [codegen id : 44]
+(71) HashAggregate [codegen id : 38]
Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51]
-(78) Filter [codegen id : 44]
+(72) Filter [codegen id : 38]
Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51]
Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
-(79) Scan parquet spark_catalog.default.catalog_sales
+(73) Scan parquet spark_catalog.default.catalog_sales
Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57]
Batched: true
Location: InMemoryFileIndex []
@@ -516,83 +480,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_so
PushedFilters: [IsNotNull(cs_item_sk)]
ReadSchema: struct<cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2)>
-(80) ColumnarToRow [codegen id : 45]
+(74) ColumnarToRow [codegen id : 75]
Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57]
-(81) Filter [codegen id : 45]
+(75) Filter [codegen id : 75]
Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57]
Condition : isnotnull(cs_item_sk#54)
-(82) Exchange
-Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57]
-Arguments: hashpartitioning(cs_item_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=14]
-
-(83) Sort [codegen id : 46]
-Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57]
-Arguments: [cs_item_sk#54 ASC NULLS FIRST], false, 0
-
-(84) ReusedExchange [Reuses operator id: 58]
+(76) ReusedExchange [Reuses operator id: 56]
Output [1]: [ss_item_sk#35]
-(85) Sort [codegen id : 64]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(86) SortMergeJoin [codegen id : 87]
+(77) BroadcastHashJoin [codegen id : 75]
Left keys [1]: [cs_item_sk#54]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(87) ReusedExchange [Reuses operator id: 172]
+(78) ReusedExchange [Reuses operator id: 160]
Output [1]: [d_date_sk#58]
-(88) BroadcastHashJoin [codegen id : 87]
+(79) BroadcastHashJoin [codegen id : 75]
Left keys [1]: [cs_sold_date_sk#57]
Right keys [1]: [d_date_sk#58]
Join type: Inner
Join condition: None
-(89) Project [codegen id : 87]
+(80) Project [codegen id : 75]
Output [3]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56]
Input [5]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, d_date_sk#58]
-(90) ReusedExchange [Reuses operator id: 72]
+(81) ReusedExchange [Reuses operator id: 66]
Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62]
-(91) BroadcastHashJoin [codegen id : 87]
+(82) BroadcastHashJoin [codegen id : 75]
Left keys [1]: [cs_item_sk#54]
Right keys [1]: [i_item_sk#59]
Join type: Inner
Join condition: None
-(92) Project [codegen id : 87]
+(83) Project [codegen id : 75]
Output [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62]
Input [7]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62]
-(93) HashAggregate [codegen id : 87]
+(84) HashAggregate [codegen id : 75]
Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62]
Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62]
Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)]
Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65]
Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68]
-(94) Exchange
+(85) Exchange
Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68]
-Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=15]
+Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=12]
-(95) HashAggregate [codegen id : 88]
+(86) HashAggregate [codegen id : 76]
Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68]
Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62]
Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)]
Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69, count(1)#70]
Results [6]: [catalog AS channel#71, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69 AS sales#72, count(1)#70 AS number_sales#73]
-(96) Filter [codegen id : 88]
+(87) Filter [codegen id : 76]
Input [6]: [channel#71, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73]
Condition : (isnotnull(sales#72) AND (cast(sales#72 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
-(97) Scan parquet spark_catalog.default.web_sales
+(88) Scan parquet spark_catalog.default.web_sales
Output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77]
Batched: true
Location: InMemoryFileIndex []
@@ -600,455 +552,443 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_so
PushedFilters: [IsNotNull(ws_item_sk)]
ReadSchema: struct<ws_item_sk:int,ws_quantity:int,ws_list_price:decimal(7,2)>
-(98) ColumnarToRow [codegen id : 89]
+(89) ColumnarToRow [codegen id : 113]
Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77]
-(99) Filter [codegen id : 89]
+(90) Filter [codegen id : 113]
Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77]
Condition : isnotnull(ws_item_sk#74)
-(100) Exchange
-Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77]
-Arguments: hashpartitioning(ws_item_sk#74, 5), ENSURE_REQUIREMENTS, [plan_id=16]
-
-(101) Sort [codegen id : 90]
-Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77]
-Arguments: [ws_item_sk#74 ASC NULLS FIRST], false, 0
-
-(102) ReusedExchange [Reuses operator id: 58]
+(91) ReusedExchange [Reuses operator id: 56]
Output [1]: [ss_item_sk#35]
-(103) Sort [codegen id : 108]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(104) SortMergeJoin [codegen id : 131]
+(92) BroadcastHashJoin [codegen id : 113]
Left keys [1]: [ws_item_sk#74]
Right keys [1]: [ss_item_sk#35]
Join type: LeftSemi
Join condition: None
-(105) ReusedExchange [Reuses operator id: 172]
+(93) ReusedExchange [Reuses operator id: 160]
Output [1]: [d_date_sk#78]
-(106) BroadcastHashJoin [codegen id : 131]
+(94) BroadcastHashJoin [codegen id : 113]
Left keys [1]: [ws_sold_date_sk#77]
Right keys [1]: [d_date_sk#78]
Join type: Inner
Join condition: None
-(107) Project [codegen id : 131]
+(95) Project [codegen id : 113]
Output [3]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76]
Input [5]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, d_date_sk#78]
-(108) ReusedExchange [Reuses operator id: 72]
+(96) ReusedExchange [Reuses operator id: 66]
Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82]
-(109) BroadcastHashJoin [codegen id : 131]
+(97) BroadcastHashJoin [codegen id : 113]
Left keys [1]: [ws_item_sk#74]
Right keys [1]: [i_item_sk#79]
Join type: Inner
Join condition: None
-(110) Project [codegen id : 131]
+(98) Project [codegen id : 113]
Output [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#80, i_class_id#81, i_category_id#82]
Input [7]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82]
-(111) HashAggregate [codegen id : 131]
+(99) HashAggregate [codegen id : 113]
Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#80, i_class_id#81, i_category_id#82]
Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82]
Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)]
Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85]
Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88]
-(112) Exchange
+(100) Exchange
Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88]
-Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [plan_id=17]
+Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [plan_id=13]
-(113) HashAggregate [codegen id : 132]
+(101) HashAggregate [codegen id : 114]
Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88]
Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82]
Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)]
Aggregate Attributes [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89, count(1)#90]
Results [6]: [web AS channel#91, i_brand_id#80, i_class_id#81, i_category_id#82, sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89 AS sales#92, count(1)#90 AS number_sales#93]
-(114) Filter [codegen id : 132]
+(102) Filter [codegen id : 114]
Input [6]: [channel#91, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93]
Condition : (isnotnull(sales#92) AND (cast(sales#92 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
-(115) Union
+(103) Union
-(116) HashAggregate [codegen id : 133]
+(104) HashAggregate [codegen id : 115]
Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51]
Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40]
Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)]
Aggregate Attributes [3]: [sum#94, isEmpty#95, sum#96]
Results [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
-(117) Exchange
+(105) Exchange
Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
-Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=18]
+Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=14]
-(118) HashAggregate [codegen id : 134]
+(106) HashAggregate [codegen id : 116]
Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
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)#100, sum(number_sales#51)#101]
Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103]
-(119) ReusedExchange [Reuses operator id: 117]
+(107) ReusedExchange [Reuses operator id: 105]
Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
-(120) HashAggregate [codegen id : 268]
+(108) HashAggregate [codegen id : 232]
Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
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)#100, sum(number_sales#51)#101]
Results [5]: [channel#49, i_brand_id#38, i_class_id#39, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103]
-(121) HashAggregate [codegen id : 268]
+(109) HashAggregate [codegen id : 232]
Input [5]: [channel#49, i_brand_id#38, i_class_id#39, sum_sales#102, number_sales#103]
Keys [3]: [channel#49, i_brand_id#38, i_class_id#39]
Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)]
Aggregate Attributes [3]: [sum#104, isEmpty#105, sum#106]
Results [6]: [channel#49, i_brand_id#38, i_class_id#39, sum#107, isEmpty#108, sum#109]
-(122) Exchange
+(110) Exchange
Input [6]: [channel#49, i_brand_id#38, i_class_id#39, sum#107, isEmpty#108, sum#109]
-Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=19]
+Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=15]
-(123) HashAggregate [codegen id : 269]
+(111) HashAggregate [codegen id : 233]
Input [6]: [channel#49, i_brand_id#38, i_class_id#39, sum#107, isEmpty#108, sum#109]
Keys [3]: [channel#49, i_brand_id#38, i_class_id#39]
Functions [2]: [sum(sum_sales#102), sum(number_sales#103)]
Aggregate Attributes [2]: [sum(sum_sales#102)#110, sum(number_sales#103)#111]
Results [6]: [channel#49, i_brand_id#38, i_class_id#39, null AS i_category_id#112, sum(sum_sales#102)#110 AS sum(sum_sales)#113, sum(number_sales#103)#111 AS sum(number_sales)#114]
-(124) ReusedExchange [Reuses operator id: 117]
+(112) ReusedExchange [Reuses operator id: 105]
Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
-(125) HashAggregate [codegen id : 403]
+(113) HashAggregate [codegen id : 349]
Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
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)#100, sum(number_sales#51)#101]
Results [4]: [channel#49, i_brand_id#38, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103]
-(126) HashAggregate [codegen id : 403]
+(114) HashAggregate [codegen id : 349]
Input [4]: [channel#49, i_brand_id#38, sum_sales#102, number_sales#103]
Keys [2]: [channel#49, i_brand_id#38]
Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)]
Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117]
Results [5]: [channel#49, i_brand_id#38, sum#118, isEmpty#119, sum#120]
-(127) Exchange
+(115) Exchange
Input [5]: [channel#49, i_brand_id#38, sum#118, isEmpty#119, sum#120]
-Arguments: hashpartitioning(channel#49, i_brand_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=20]
+Arguments: hashpartitioning(channel#49, i_brand_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=16]
-(128) HashAggregate [codegen id : 404]
+(116) HashAggregate [codegen id : 350]
Input [5]: [channel#49, i_brand_id#38, sum#118, isEmpty#119, sum#120]
Keys [2]: [channel#49, i_brand_id#38]
Functions [2]: [sum(sum_sales#102), sum(number_sales#103)]
Aggregate Attributes [2]: [sum(sum_sales#102)#121, sum(number_sales#103)#122]
Results [6]: [channel#49, i_brand_id#38, null AS i_class_id#123, null AS i_category_id#124, sum(sum_sales#102)#121 AS sum(sum_sales)#125, sum(number_sales#103)#122 AS sum(number_sales)#126]
-(129) ReusedExchange [Reuses operator id: 117]
+(117) ReusedExchange [Reuses operator id: 105]
Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
-(130) HashAggregate [codegen id : 538]
+(118) HashAggregate [codegen id : 466]
Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
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)#100, sum(number_sales#51)#101]
Results [3]: [channel#49, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103]
-(131) HashAggregate [codegen id : 538]
+(119) HashAggregate [codegen id : 466]
Input [3]: [channel#49, sum_sales#102, number_sales#103]
Keys [1]: [channel#49]
Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)]
Aggregate Attributes [3]: [sum#127, isEmpty#128, sum#129]
Results [4]: [channel#49, sum#130, isEmpty#131, sum#132]
-(132) Exchange
+(120) Exchange
Input [4]: [channel#49, sum#130, isEmpty#131, sum#132]
-Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, [plan_id=21]
+Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, [plan_id=17]
-(133) HashAggregate [codegen id : 539]
+(121) HashAggregate [codegen id : 467]
Input [4]: [channel#49, sum#130, isEmpty#131, sum#132]
Keys [1]: [channel#49]
Functions [2]: [sum(sum_sales#102), sum(number_sales#103)]
Aggregate Attributes [2]: [sum(sum_sales#102)#133, sum(number_sales#103)#134]
Results [6]: [channel#49, null AS i_brand_id#135, null AS i_class_id#136, null AS i_category_id#137, sum(sum_sales#102)#133 AS sum(sum_sales)#138, sum(number_sales#103)#134 AS sum(number_sales)#139]
-(134) ReusedExchange [Reuses operator id: 117]
+(122) ReusedExchange [Reuses operator id: 105]
Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
-(135) HashAggregate [codegen id : 673]
+(123) HashAggregate [codegen id : 583]
Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
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)#100, sum(number_sales#51)#101]
Results [2]: [sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103]
-(136) HashAggregate [codegen id : 673]
+(124) HashAggregate [codegen id : 583]
Input [2]: [sum_sales#102, number_sales#103]
Keys: []
Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)]
Aggregate Attributes [3]: [sum#140, isEmpty#141, sum#142]
Results [3]: [sum#143, isEmpty#144, sum#145]
-(137) Exchange
+(125) Exchange
Input [3]: [sum#143, isEmpty#144, sum#145]
-Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=22]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18]
-(138) HashAggregate [codegen id : 674]
+(126) HashAggregate [codegen id : 584]
Input [3]: [sum#143, isEmpty#144, sum#145]
Keys: []
Functions [2]: [sum(sum_sales#102), sum(number_sales#103)]
Aggregate Attributes [2]: [sum(sum_sales#102)#146, sum(number_sales#103)#147]
Results [6]: [null AS channel#148, null AS i_brand_id#149, null AS i_class_id#150, null AS i_category_id#151, sum(sum_sales#102)#146 AS sum(sum_sales)#152, sum(number_sales#103)#147 AS sum(number_sales)#153]
-(139) Union
+(127) Union
-(140) HashAggregate [codegen id : 675]
+(128) HashAggregate [codegen id : 585]
Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
Keys [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
-(141) Exchange
+(129) Exchange
Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
-Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103, 5), ENSURE_REQUIREMENTS, [plan_id=23]
+Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103, 5), ENSURE_REQUIREMENTS, [plan_id=19]
-(142) HashAggregate [codegen id : 676]
+(130) HashAggregate [codegen id : 586]
Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
Keys [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
-(143) TakeOrderedAndProject
+(131) TakeOrderedAndProject
Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
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#102, number_sales#103]
===== Subqueries =====
-Subquery:1 Hosting operator id = 78 Hosting Expression = Subquery scalar-subquery#52, [id=#53]
-* HashAggregate (162)
-+- Exchange (161)
- +- * HashAggregate (160)
- +- Union (159)
- :- * Project (148)
- : +- * BroadcastHashJoin Inner BuildRight (147)
- : :- * ColumnarToRow (145)
- : : +- Scan parquet spark_catalog.default.store_sales (144)
- : +- ReusedExchange (146)
- :- * Project (153)
- : +- * BroadcastHashJoin Inner BuildRight (152)
- : :- * ColumnarToRow (150)
- : : +- Scan parquet spark_catalog.default.catalog_sales (149)
- : +- ReusedExchange (151)
- +- * Project (158)
- +- * BroadcastHashJoin Inner BuildRight (157)
- :- * ColumnarToRow (155)
- : +- Scan parquet spark_catalog.default.web_sales (154)
- +- ReusedExchange (156)
-
-
-(144) Scan parquet spark_catalog.default.store_sales
+Subquery:1 Hosting operator id = 72 Hosting Expression = Subquery scalar-subquery#52, [id=#53]
+* 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#154, ss_list_price#155, ss_sold_date_sk#156]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ss_sold_date_sk#156), dynamicpruningexpression(ss_sold_date_sk#156 IN dynamicpruning#12)]
ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)>
-(145) ColumnarToRow [codegen id : 2]
+(133) ColumnarToRow [codegen id : 2]
Input [3]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156]
-(146) ReusedExchange [Reuses operator id: 177]
+(134) ReusedExchange [Reuses operator id: 165]
Output [1]: [d_date_sk#157]
-(147) BroadcastHashJoin [codegen id : 2]
+(135) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [ss_sold_date_sk#156]
Right keys [1]: [d_date_sk#157]
Join type: Inner
Join condition: None
-(148) Project [codegen id : 2]
+(136) Project [codegen id : 2]
Output [2]: [ss_quantity#154 AS quantity#158, ss_list_price#155 AS list_price#159]
Input [4]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156, d_date_sk#157]
-(149) Scan parquet spark_catalog.default.catalog_sales
+(137) Scan parquet spark_catalog.default.catalog_sales
Output [3]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(cs_sold_date_sk#162), dynamicpruningexpression(cs_sold_date_sk#162 IN dynamicpruning#163)]
ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)>
-(150) ColumnarToRow [codegen id : 4]
+(138) ColumnarToRow [codegen id : 4]
Input [3]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162]
-(151) ReusedExchange [Reuses operator id: 167]
+(139) ReusedExchange [Reuses operator id: 155]
Output [1]: [d_date_sk#164]
-(152) BroadcastHashJoin [codegen id : 4]
+(140) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [cs_sold_date_sk#162]
Right keys [1]: [d_date_sk#164]
Join type: Inner
Join condition: None
-(153) Project [codegen id : 4]
+(141) Project [codegen id : 4]
Output [2]: [cs_quantity#160 AS quantity#165, cs_list_price#161 AS list_price#166]
Input [4]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162, d_date_sk#164]
-(154) Scan parquet spark_catalog.default.web_sales
+(142) Scan parquet spark_catalog.default.web_sales
Output [3]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ws_sold_date_sk#169), dynamicpruningexpression(ws_sold_date_sk#169 IN dynamicpruning#163)]
ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)>
-(155) ColumnarToRow [codegen id : 6]
+(143) ColumnarToRow [codegen id : 6]
Input [3]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169]
-(156) ReusedExchange [Reuses operator id: 167]
+(144) ReusedExchange [Reuses operator id: 155]
Output [1]: [d_date_sk#170]
-(157) BroadcastHashJoin [codegen id : 6]
+(145) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [ws_sold_date_sk#169]
Right keys [1]: [d_date_sk#170]
Join type: Inner
Join condition: None
-(158) Project [codegen id : 6]
+(146) Project [codegen id : 6]
Output [2]: [ws_quantity#167 AS quantity#171, ws_list_price#168 AS list_price#172]
Input [4]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169, d_date_sk#170]
-(159) Union
+(147) Union
-(160) HashAggregate [codegen id : 7]
+(148) HashAggregate [codegen id : 7]
Input [2]: [quantity#158, list_price#159]
Keys: []
Functions [1]: [partial_avg((cast(quantity#158 as decimal(10,0)) * list_price#159))]
Aggregate Attributes [2]: [sum#173, count#174]
Results [2]: [sum#175, count#176]
-(161) Exchange
+(149) Exchange
Input [2]: [sum#175, count#176]
-Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=24]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=20]
-(162) HashAggregate [codegen id : 8]
+(150) HashAggregate [codegen id : 8]
Input [2]: [sum#175, count#176]
Keys: []
Functions [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))]
Aggregate Attributes [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177]
Results [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177 AS average_sales#178]
-Subquery:2 Hosting operator id = 144 Hosting Expression = ss_sold_date_sk#156 IN dynamicpruning#12
+Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#156 IN dynamicpruning#12
-Subquery:3 Hosting operator id = 149 Hosting Expression = cs_sold_date_sk#162 IN dynamicpruning#163
-BroadcastExchange (167)
-+- * Project (166)
- +- * Filter (165)
- +- * ColumnarToRow (164)
- +- Scan parquet spark_catalog.default.date_dim (163)
+Subquery:3 Hosting operator id = 137 Hosting Expression = cs_sold_date_sk#162 IN dynamicpruning#163
+BroadcastExchange (155)
++- * Project (154)
+ +- * Filter (153)
+ +- * ColumnarToRow (152)
+ +- Scan parquet spark_catalog.default.date_dim (151)
-(163) Scan parquet spark_catalog.default.date_dim
+(151) Scan parquet spark_catalog.default.date_dim
Output [2]: [d_date_sk#164, d_year#179]
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<d_date_sk:int,d_year:int>
-(164) ColumnarToRow [codegen id : 1]
+(152) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#164, d_year#179]
-(165) Filter [codegen id : 1]
+(153) Filter [codegen id : 1]
Input [2]: [d_date_sk#164, d_year#179]
Condition : (((isnotnull(d_year#179) AND (d_year#179 >= 1998)) AND (d_year#179 <= 2000)) AND isnotnull(d_date_sk#164))
-(166) Project [codegen id : 1]
+(154) Project [codegen id : 1]
Output [1]: [d_date_sk#164]
Input [2]: [d_date_sk#164, d_year#179]
-(167) BroadcastExchange
+(155) BroadcastExchange
Input [1]: [d_date_sk#164]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=25]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21]
-Subquery:4 Hosting operator id = 154 Hosting Expression = ws_sold_date_sk#169 IN dynamicpruning#163
+Subquery:4 Hosting operator id = 142 Hosting Expression = ws_sold_date_sk#169 IN dynamicpruning#163
Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (172)
-+- * Project (171)
- +- * Filter (170)
- +- * ColumnarToRow (169)
- +- Scan parquet spark_catalog.default.date_dim (168)
+BroadcastExchange (160)
++- * Project (159)
+ +- * Filter (158)
+ +- * ColumnarToRow (157)
+ +- Scan parquet spark_catalog.default.date_dim (156)
-(168) Scan parquet spark_catalog.default.date_dim
+(156) Scan parquet spark_catalog.default.date_dim
Output [3]: [d_date_sk#36, d_year#180, d_moy#181]
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<d_date_sk:int,d_year:int,d_moy:int>
-(169) ColumnarToRow [codegen id : 1]
+(157) ColumnarToRow [codegen id : 1]
Input [3]: [d_date_sk#36, d_year#180, d_moy#181]
-(170) Filter [codegen id : 1]
+(158) Filter [codegen id : 1]
Input [3]: [d_date_sk#36, d_year#180, d_moy#181]
Condition : ((((isnotnull(d_year#180) AND isnotnull(d_moy#181)) AND (d_year#180 = 2000)) AND (d_moy#181 = 11)) AND isnotnull(d_date_sk#36))
-(171) Project [codegen id : 1]
+(159) Project [codegen id : 1]
Output [1]: [d_date_sk#36]
Input [3]: [d_date_sk#36, d_year#180, d_moy#181]
-(172) BroadcastExchange
+(160) BroadcastExchange
Input [1]: [d_date_sk#36]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=26]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=22]
-Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
-BroadcastExchange (177)
-+- * Project (176)
- +- * Filter (175)
- +- * ColumnarToRow (174)
- +- Scan parquet spark_catalog.default.date_dim (173)
+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)
-(173) Scan parquet spark_catalog.default.date_dim
+(161) Scan parquet spark_catalog.default.date_dim
Output [2]: [d_date_sk#13, d_year#182]
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<d_date_sk:int,d_year:int>
-(174) ColumnarToRow [codegen id : 1]
+(162) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#13, d_year#182]
-(175) Filter [codegen id : 1]
+(163) Filter [codegen id : 1]
Input [2]: [d_date_sk#13, d_year#182]
Condition : (((isnotnull(d_year#182) AND (d_year#182 >= 1999)) AND (d_year#182 <= 2001)) AND isnotnull(d_date_sk#13))
-(176) Project [codegen id : 1]
+(164) Project [codegen id : 1]
Output [1]: [d_date_sk#13]
Input [2]: [d_date_sk#13, d_year#182]
-(177) BroadcastExchange
+(165) BroadcastExchange
Input [1]: [d_date_sk#13]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=27]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=23]
-Subquery:7 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
+Subquery:7 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
-Subquery:8 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
+Subquery:8 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
-Subquery:9 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
+Subquery:9 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
-Subquery:10 Hosting operator id = 79 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5
+Subquery:10 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5
-Subquery:11 Hosting operator id = 114 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
+Subquery:11 Hosting operator id = 102 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
-Subquery:12 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5
+Subquery:12 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5
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 c0788a25e93..84c70b046c7 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,27 +1,27 @@
TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales]
- WholeStageCodegen (676)
+ WholeStageCodegen (586)
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 (675)
+ WholeStageCodegen (585)
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales]
InputAdapter
Union
- WholeStageCodegen (134)
+ WholeStageCodegen (116)
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 (133)
+ 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 (44)
+ 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 #18
+ Exchange #16
WholeStageCodegen (7)
HashAggregate [quantity,list_price] [sum,count,sum,count]
InputAdapter
@@ -34,7 +34,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
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] #10
+ ReusedExchange [d_date_sk] #9
WholeStageCodegen (4)
Project [cs_quantity,cs_list_price]
BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
@@ -42,7 +42,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #4
- BroadcastExchange #19
+ BroadcastExchange #17
WholeStageCodegen (1)
Project [d_date_sk]
Filter [d_year,d_date_sk]
@@ -50,7 +50,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
- ReusedExchange [d_date_sk] #19
+ ReusedExchange [d_date_sk] #17
WholeStageCodegen (6)
Project [ws_quantity,ws_list_price]
BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
@@ -59,256 +59,220 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
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] #19
+ 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 (43)
+ 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]
- SortMergeJoin [ss_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (2)
- Sort [ss_item_sk]
- InputAdapter
- Exchange [ss_item_sk] #4
- WholeStageCodegen (1)
- 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 #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]
- InputAdapter
- WholeStageCodegen (20)
- Sort [ss_item_sk]
- InputAdapter
- Exchange [ss_item_sk] #6
- WholeStageCodegen (19)
- 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]
+ 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.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
- InputAdapter
- BroadcastExchange #7
- WholeStageCodegen (18)
- 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 #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]
+ 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]
+ InputAdapter
+ WholeStageCodegen (11)
+ Sort [brand_id,class_id,category_id]
InputAdapter
- WholeStageCodegen (13)
- Sort [brand_id,class_id,category_id]
- InputAdapter
- Exchange [brand_id,class_id,category_id] #8
- WholeStageCodegen (12)
- HashAggregate [brand_id,class_id,category_id]
- InputAdapter
- Exchange [brand_id,class_id,category_id] #9
- WholeStageCodegen (11)
- 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 #10
- 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] #10
- InputAdapter
- BroadcastExchange #11
- WholeStageCodegen (10)
- 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] #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]
+ InputAdapter
+ WholeStageCodegen (3)
+ Sort [i_brand_id,i_class_id,i_category_id]
InputAdapter
- WholeStageCodegen (5)
- Sort [i_brand_id,i_class_id,i_category_id]
- InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #12
- WholeStageCodegen (4)
- 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] #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]
+ InputAdapter
+ WholeStageCodegen (7)
+ Sort [i_brand_id,i_class_id,i_category_id]
InputAdapter
- WholeStageCodegen (9)
- Sort [i_brand_id,i_class_id,i_category_id]
- InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #13
- WholeStageCodegen (8)
- 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] #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
+ InputAdapter
+ ReusedExchange [d_date_sk] #9
+ InputAdapter
+ BroadcastExchange #13
+ WholeStageCodegen (5)
+ Filter [i_item_sk]
+ ColumnarToRow
InputAdapter
- ReusedExchange [d_date_sk] #10
- InputAdapter
- BroadcastExchange #14
- WholeStageCodegen (7)
- 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 (17)
- Sort [i_brand_id,i_class_id,i_category_id]
- InputAdapter
- Exchange [i_brand_id,i_class_id,i_category_id] #15
- WholeStageCodegen (16)
- 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] #10
- InputAdapter
- ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14
+ 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] #5
+ ReusedExchange [d_date_sk] #4
InputAdapter
- BroadcastExchange #16
- WholeStageCodegen (42)
- SortMergeJoin [i_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (23)
- Sort [i_item_sk]
- InputAdapter
- Exchange [i_item_sk] #17
- WholeStageCodegen (22)
- Filter [i_item_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ 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
- WholeStageCodegen (41)
- Sort [ss_item_sk]
- InputAdapter
- ReusedExchange [ss_item_sk] #6
- WholeStageCodegen (88)
+ 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] #20
- WholeStageCodegen (87)
+ 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]
- SortMergeJoin [cs_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (46)
- Sort [cs_item_sk]
- InputAdapter
- Exchange [cs_item_sk] #21
- WholeStageCodegen (45)
- 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
+ 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
InputAdapter
- WholeStageCodegen (64)
- Sort [ss_item_sk]
- InputAdapter
- ReusedExchange [ss_item_sk] #6
+ ReusedExchange [ss_item_sk] #5
InputAdapter
- ReusedExchange [d_date_sk] #5
+ ReusedExchange [d_date_sk] #4
InputAdapter
- ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16
- WholeStageCodegen (132)
+ 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] #22
- WholeStageCodegen (131)
+ 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]
- SortMergeJoin [ws_item_sk,ss_item_sk]
- InputAdapter
- WholeStageCodegen (90)
- Sort [ws_item_sk]
- InputAdapter
- Exchange [ws_item_sk] #23
- WholeStageCodegen (89)
- 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
+ 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
- WholeStageCodegen (108)
- Sort [ss_item_sk]
- InputAdapter
- ReusedExchange [ss_item_sk] #6
+ ReusedExchange [ss_item_sk] #5
InputAdapter
- ReusedExchange [d_date_sk] #5
+ ReusedExchange [d_date_sk] #4
InputAdapter
- ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16
- WholeStageCodegen (269)
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15
+ WholeStageCodegen (233)
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] #24
- WholeStageCodegen (268)
+ Exchange [channel,i_brand_id,i_class_id] #20
+ WholeStageCodegen (232)
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 (404)
+ WholeStageCodegen (350)
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] #25
- WholeStageCodegen (403)
+ Exchange [channel,i_brand_id] #21
+ WholeStageCodegen (349)
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 (539)
+ WholeStageCodegen (467)
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] #26
- WholeStageCodegen (538)
+ Exchange [channel] #22
+ WholeStageCodegen (466)
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 (674)
+ WholeStageCodegen (584)
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 #27
- WholeStageCodegen (673)
+ Exchange #23
+ WholeStageCodegen (583)
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
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org