From 69be17bfc3845668fd8f5aed8026c04da4496677 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 26 Jul 2020 18:25:24 +0800 Subject: [PATCH 1/9] Infer filters from DPP --- .../sql/catalyst/expressions/subquery.scala | 4 +- .../spark/sql/execution/SparkOptimizer.scala | 5 +- .../sql/DynamicPartitionPruningSuite.scala | 51 +++++++++++++++++++ 3 files changed, 57 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index ff8856708c6d1..681163ca507b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -74,12 +74,14 @@ object SubqueryExpression { } /** - * Returns true when an expression contains a subquery that has outer reference(s). The outer + * Returns true when an expression contains a subquery that has outer reference(s) except + * the [[org.apache.spark.sql.catalyst.expressions.DynamicPruningSubquery]]. The outer * reference attributes are kept as children of subquery expression by * [[org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveSubquery]] */ def hasCorrelatedSubquery(e: Expression): Boolean = { e.find { + case _: DynamicPruningSubquery => false case s: SubqueryExpression => s.children.nonEmpty case _ => false }.isDefined diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 33b86a2b5340c..726043f8cd602 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -44,8 +44,9 @@ class SparkOptimizer( Batch("PartitionPruning", Once, PartitionPruning, OptimizeSubqueries) :+ - Batch("Pushdown Filters from PartitionPruning", fixedPoint, - PushDownPredicates) :+ + Batch("Pushdown and infer Filters from PartitionPruning", fixedPoint, + PushDownPredicates, + InferFiltersFromConstraints) :+ Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, PruneFilters)) ++ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index cdf9ea4b31ee7..959b78e4d714e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -1280,6 +1280,57 @@ abstract class DynamicPartitionPruningSuiteBase ) } } + + test("Infer DPP filter to other partition column") { + val df = sql( + """ + |SELECT t11.store_id, + | t11.code, + | t3.product_id + |FROM (SELECT t1.store_id, + | t2.code + | FROM fact_stats t1 + | JOIN code_stats t2 + | ON t1.store_id = t2.store_id) t11 + | JOIN product t3 + | ON t11.store_id = t3.store_id AND t3.product_id < 3 + |""".stripMargin) + + assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 2) + checkDistinctSubqueries(df, 1) + checkPartitionPruningPredicate(df, false, true) + + checkAnswer(df, + Row(2, 20, 2) :: + Row(2, 20, 2) :: + Row(1, 10, 1) :: Nil + ) + } + + test("Should not infer DPP filter to other non-partition column") { + val df = sql( + """ + |SELECT t11.store_id, + | t11.country, + | t3.product_id + |FROM (SELECT t1.store_id, + | t2.country + | FROM fact_stats t1 + | JOIN dim_stats t2 + | ON t1.store_id = t2.store_id) t11 + | JOIN product t3 + | ON t11.store_id = t3.store_id AND t3.product_id < 3 + |""".stripMargin) + + assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 1) + checkPartitionPruningPredicate(df, false, true) + + checkAnswer(df, + Row(2, "NL", 2) :: + Row(2, "NL", 2) :: + Row(1, "NL", 1) :: Nil + ) + } } class DynamicPartitionPruningSuiteAEOff extends DynamicPartitionPruningSuiteBase { From f001b277fbc241d1534f905ea1d9e24e21e7f290 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 26 Jul 2020 23:25:16 +0800 Subject: [PATCH 2/9] Change strategy to Once --- .../org/apache/spark/sql/execution/SparkOptimizer.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 726043f8cd602..b547a7c64100e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -44,8 +44,9 @@ class SparkOptimizer( Batch("PartitionPruning", Once, PartitionPruning, OptimizeSubqueries) :+ - Batch("Pushdown and infer Filters from PartitionPruning", fixedPoint, - PushDownPredicates, + Batch("Pushdown Filters from PartitionPruning", fixedPoint, + PushDownPredicates) :+ + Batch("Infer Filters from DynamicPruning", Once, InferFiltersFromConstraints) :+ Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, From bcc81be47ea0d8f04a3d508162d883bc57ecd68e Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 27 Jul 2020 19:22:14 +0800 Subject: [PATCH 3/9] Fix Once strategy's idempotence is broken for batch Infer Filters --- .../org/apache/spark/sql/catalyst/optimizer/Optimizer.scala | 1 + .../scala/org/apache/spark/sql/execution/SparkOptimizer.scala | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index b8da954d938c4..e7fa5201ab87d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -51,6 +51,7 @@ abstract class Optimizer(catalogManager: CatalogManager) override protected val excludedOnceBatches: Set[String] = Set( "PartitionPruning", + "Infer Filters from PartitionPruning", "Extract Python UDFs") protected def fixedPoint = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index b547a7c64100e..1eec4495b1805 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -46,7 +46,7 @@ class SparkOptimizer( OptimizeSubqueries) :+ Batch("Pushdown Filters from PartitionPruning", fixedPoint, PushDownPredicates) :+ - Batch("Infer Filters from DynamicPruning", Once, + Batch("Infer Filters from PartitionPruning", Once, InferFiltersFromConstraints) :+ Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, From 98f7275338bd6bedcd20f3c21e3858fd704ebb58 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 21 Aug 2020 16:00:05 +0800 Subject: [PATCH 4/9] Remove any Filters with DynamicPruning that didn't filter on partition column --- .../spark/sql/execution/SparkOptimizer.scala | 4 +++- .../CleanupDynamicPruningFilters.scala | 19 ++++++++++++++++++- .../dynamicpruning/PartitionPruning.scala | 2 +- .../sql/DynamicPartitionPruningSuite.scala | 2 ++ 4 files changed, 24 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 1eec4495b1805..d62a950aff2bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -44,10 +44,12 @@ class SparkOptimizer( Batch("PartitionPruning", Once, PartitionPruning, OptimizeSubqueries) :+ - Batch("Pushdown Filters from PartitionPruning", fixedPoint, + Batch("Pushdown Filters from PartitionPruning before Inferring Filters", fixedPoint, PushDownPredicates) :+ Batch("Infer Filters from PartitionPruning", Once, InferFiltersFromConstraints) :+ + Batch("Pushdown Filters from PartitionPruning after Inferring Filters", fixedPoint, + PushDownPredicates) :+ Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, PruneFilters)) ++ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/CleanupDynamicPruningFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/CleanupDynamicPruningFilters.scala index 1cf55a2f2954d..5af6436033e56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/CleanupDynamicPruningFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/CleanupDynamicPruningFilters.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.dynamicpruning -import org.apache.spark.sql.catalyst.expressions.{DynamicPruning, PredicateHelper} +import org.apache.spark.sql.catalyst.expressions.{DynamicPruning, DynamicPruningSubquery, Expression, PredicateHelper} import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan} @@ -32,12 +32,29 @@ import org.apache.spark.sql.internal.SQLConf */ object CleanupDynamicPruningFilters extends Rule[LogicalPlan] with PredicateHelper { + private def isFilterOnNonPartition(condition: Expression, child: LogicalPlan): Boolean = { + splitConjunctivePredicates(condition).exists { + case DynamicPruningSubquery(pruningKey, _, _, _, _, _) => + PartitionPruning.getPartitionTableScan(pruningKey, child).isEmpty + case _ => false + } + } + override def apply(plan: LogicalPlan): LogicalPlan = { if (!SQLConf.get.dynamicPartitionPruningEnabled) { return plan } plan.transform { + // Remove any Filters with DynamicPruning that didn't filter on partition column`. + // This is inferred by Infer Filters from PartitionPruning. + case f @ Filter(condition, child) if isFilterOnNonPartition(condition, child) => + val newCondition = condition.transform { + case DynamicPruningSubquery(pruningKey, _, _, _, _, _) + if PartitionPruning.getPartitionTableScan(pruningKey, child).isEmpty => + TrueLiteral + } + f.copy(condition = newCondition) // pass through anything that is pushed down into PhysicalOperation case p @ PhysicalOperation(_, _, LogicalRelation(_: HadoopFsRelation, _, _, _)) => p // remove any Filters with DynamicPruning that didn't get pushed down to PhysicalOperation. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala index 43c6581632687..34a5b65a58903 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala @@ -147,7 +147,7 @@ object PartitionPruning extends Rule[LogicalPlan] with PredicateHelper { } // the pruning overhead is the total size in bytes of all scan relations - val overhead = otherPlan.collectLeaves().map(_.stats.sizeInBytes).sum.toFloat + val overhead = otherPlan.collectLeaves().map(_.stats.sizeInBytes).sum filterRatio * partPlan.stats.sizeInBytes.toFloat > overhead.toFloat } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index 959b78e4d714e..3f1e3e6c92ed2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -1299,6 +1299,7 @@ abstract class DynamicPartitionPruningSuiteBase assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 2) checkDistinctSubqueries(df, 1) checkPartitionPruningPredicate(df, false, true) + assert(!checkUnpushedFilters(df), "Inferred dynamic pruning expression has been pushed down.") checkAnswer(df, Row(2, 20, 2) :: @@ -1324,6 +1325,7 @@ abstract class DynamicPartitionPruningSuiteBase assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 1) checkPartitionPruningPredicate(df, false, true) + assert(!checkUnpushedFilters(df), "Inferred dynamic pruning expression should be removed.") checkAnswer(df, Row(2, "NL", 2) :: From 4901af4ac5aa8a5b6130899cfd19c68622d7cf2f Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 24 Aug 2020 14:37:10 +0800 Subject: [PATCH 5/9] Remove no benefit inferred DPP on partition column when ${SQLConf.EXCHANGE_REUSE_ENABLED.key} is disabled --- .../spark/sql/execution/SparkOptimizer.scala | 1 + .../CleanupDynamicPruningFilters.scala | 22 ++- .../dynamicpruning/PartitionPruning.scala | 2 +- .../sql/DynamicPartitionPruningSuite.scala | 178 +++++++++++++----- 4 files changed, 145 insertions(+), 58 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index d62a950aff2bc..a7ac92404a057 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -52,6 +52,7 @@ class SparkOptimizer( PushDownPredicates) :+ Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, + BooleanSimplification, PruneFilters)) ++ postHocOptimizationBatches :+ Batch("Extract Python UDFs", Once, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/CleanupDynamicPruningFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/CleanupDynamicPruningFilters.scala index 5af6436033e56..c9029b4ad60b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/CleanupDynamicPruningFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/CleanupDynamicPruningFilters.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.dynamicpruning.PartitionPruning._ import org.apache.spark.sql.internal.SQLConf /** @@ -32,10 +33,12 @@ import org.apache.spark.sql.internal.SQLConf */ object CleanupDynamicPruningFilters extends Rule[LogicalPlan] with PredicateHelper { - private def isFilterOnNonPartition(condition: Expression, child: LogicalPlan): Boolean = { + // Check whether need to remove inferred DPP. + private def isRemoveInferred(condition: Expression, child: LogicalPlan): Boolean = { splitConjunctivePredicates(condition).exists { - case DynamicPruningSubquery(pruningKey, _, _, _, _, _) => - PartitionPruning.getPartitionTableScan(pruningKey, child).isEmpty + case DynamicPruningSubquery(pruningKey, buildQuery, buildKeys, index, _, _) => + getPartitionTableScan(pruningKey, child).isEmpty || (!SQLConf.get.exchangeReuseEnabled && + !pruningHasBenefit(pruningKey, child, buildKeys(index), buildQuery)) case _ => false } } @@ -46,12 +49,17 @@ object CleanupDynamicPruningFilters extends Rule[LogicalPlan] with PredicateHelp } plan.transform { - // Remove any Filters with DynamicPruning that didn't filter on partition column`. - // This is inferred by Infer Filters from PartitionPruning. - case f @ Filter(condition, child) if isFilterOnNonPartition(condition, child) => + // Remove any DynamicPruning Filters that didn't filter on partition column and + // do not have has benefit. This is inferred by Infer Filters from PartitionPruning. + case f @ Filter(condition, child) + if SQLConf.get.constraintPropagationEnabled && isRemoveInferred(condition, child) => val newCondition = condition.transform { case DynamicPruningSubquery(pruningKey, _, _, _, _, _) - if PartitionPruning.getPartitionTableScan(pruningKey, child).isEmpty => + if getPartitionTableScan(pruningKey, child).isEmpty => + TrueLiteral + case DynamicPruningSubquery(pruningKey, buildQuery, buildKeys, index, _, _) + if !SQLConf.get.exchangeReuseEnabled && + !pruningHasBenefit(pruningKey, child, buildKeys(index), buildQuery) => TrueLiteral } f.copy(condition = newCondition) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala index 34a5b65a58903..68455d8aeb53a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala @@ -111,7 +111,7 @@ object PartitionPruning extends Rule[LogicalPlan] with PredicateHelper { * using column statistics if they are available, otherwise we use the config value of * `spark.sql.optimizer.joinFilterRatio`. */ - private def pruningHasBenefit( + def pruningHasBenefit( partExpr: Expression, partPlan: LogicalPlan, otherExpr: Expression, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index 7005669cc2596..d902643ae2535 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -1305,57 +1305,135 @@ abstract class DynamicPartitionPruningSuiteBase } } - test("Infer DPP filter to other partition column") { - val df = sql( - """ - |SELECT t11.store_id, - | t11.code, - | t3.product_id - |FROM (SELECT t1.store_id, - | t2.code - | FROM fact_stats t1 - | JOIN code_stats t2 - | ON t1.store_id = t2.store_id) t11 - | JOIN product t3 - | ON t11.store_id = t3.store_id AND t3.product_id < 3 - |""".stripMargin) - - assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 2) - checkDistinctSubqueries(df, 1) - checkPartitionPruningPredicate(df, false, true) - assert(!checkUnpushedFilters(df), "Inferred dynamic pruning expression has been pushed down.") - - checkAnswer(df, - Row(2, 20, 2) :: - Row(2, 20, 2) :: - Row(1, 10, 1) :: Nil - ) - } + test("Infer filters from DPP") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + withTable("df1", "df2", "df3", "df4") { + spark.range(1000) + .select(col("id"), col("id").as("k")) + .write + .partitionBy("k") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("df1") - test("Should not infer DPP filter to other non-partition column") { - val df = sql( - """ - |SELECT t11.store_id, - | t11.country, - | t3.product_id - |FROM (SELECT t1.store_id, - | t2.country - | FROM fact_stats t1 - | JOIN dim_stats t2 - | ON t1.store_id = t2.store_id) t11 - | JOIN product t3 - | ON t11.store_id = t3.store_id AND t3.product_id < 3 - |""".stripMargin) - - assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 1) - checkPartitionPruningPredicate(df, false, true) - assert(!checkUnpushedFilters(df), "Inferred dynamic pruning expression should be removed.") - - checkAnswer(df, - Row(2, "NL", 2) :: - Row(2, "NL", 2) :: - Row(1, "NL", 1) :: Nil - ) + spark.range(1000) + .select(col("id"), col("id").as("k")) + .write + .partitionBy("k") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("df2") + + spark.range(5) + .select(col("id"), col("id").as("k")) + .write + .partitionBy("k") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("df3") + + spark.range(100) + .select(col("id"), col("id").as("k")) + .write + .format(tableFormat) + .mode("overwrite") + .saveAsTable("df4") + + spark.range(1000) + .select(col("id"), col("id").as("k")) + .write + .format(tableFormat) + .mode("overwrite") + .saveAsTable("df5") + + Given(s"Inferred DPP on partition column") + Seq(true, false).foreach { infer => + withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> s"$infer") { + val df = sql( + """ + |SELECT t1.id, + | df4.k + |FROM (SELECT df2.id, + | df1.k + | FROM df1 + | JOIN df2 + | ON df1.k = df2.k) t1 + | JOIN df4 + | ON t1.k = df4.k AND df4.id < 2 + |""".stripMargin) + if (infer) { + assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 2) + } else { + assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 1) + } + checkDistinctSubqueries(df, 1) + checkPartitionPruningPredicate(df, false, true) + assert(!checkUnpushedFilters(df)) + + checkAnswer(df, Row(0, 0) :: Row(1, 1) :: Nil) + } + } + + Given("Remove no benefit inferred DPP on partition column " + + s"when ${SQLConf.EXCHANGE_REUSE_ENABLED.key} is disabled") + Seq(true, false).foreach { reuse => + Seq(true, false).foreach { broadcastOnly => + withSQLConf( + SQLConf.EXCHANGE_REUSE_ENABLED.key -> s"$reuse", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> s"$broadcastOnly") { + val df = sql( + """ + |SELECT t1.id, + | df4.k + |FROM (SELECT df3.id, + | df1.k + | FROM df1 + | JOIN df3 + | ON df1.k = df3.k) t1 + | JOIN df4 + | ON t1.k = df4.k AND df4.id < 2 + |""".stripMargin) + + if (!reuse) { + assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 1) + checkDistinctSubqueries(df, 0) + checkPartitionPruningPredicate(df, !broadcastOnly, false) + } else { + assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 2) + checkDistinctSubqueries(df, 1) + checkPartitionPruningPredicate(df, false, true) + } + assert(!checkUnpushedFilters(df)) + + checkAnswer(df, Row(0, 0) :: Row(1, 1) :: Nil) + } + } + } + + Given("Remove inferred DPP on non-partition column") + withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "true") { + val df = sql( + """ + |SELECT t1.id, + | df4.k + |FROM (SELECT df5.id, + | df1.k + | FROM df1 + | JOIN df5 + | ON df1.k = df5.k) t1 + | JOIN df4 + | ON t1.k = df4.k AND df4.id < 2 + |""".stripMargin) + + assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 1) + checkDistinctSubqueries(df, 1) + checkPartitionPruningPredicate(df, false, true) + assert(!checkUnpushedFilters(df)) + + checkAnswer(df, Row(0, 0) :: Row(1, 1) :: Nil) + } + } + } } } From c436bc4f7d5519797ca45c1db17f150defeb73fd Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 24 Aug 2020 15:00:55 +0800 Subject: [PATCH 6/9] Update PlanStabilitySuite --- .../approved-plans-v1_4/q10.sf100/explain.txt | 8 +- .../q10.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q10/explain.txt | 8 +- .../approved-plans-v1_4/q10/simplified.txt | 4 +- .../q14a.sf100/explain.txt | 4 +- .../q14a.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q14a/explain.txt | 4 +- .../approved-plans-v1_4/q14a/simplified.txt | 2 +- .../q14b.sf100/explain.txt | 4 +- .../q14b.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q14b/explain.txt | 4 +- .../approved-plans-v1_4/q14b/simplified.txt | 2 +- .../approved-plans-v1_4/q16.sf100/explain.txt | 168 +++++++------ .../q16.sf100/simplified.txt | 16 +- .../approved-plans-v1_4/q16/explain.txt | 166 +++++++------ .../approved-plans-v1_4/q16/simplified.txt | 16 +- .../q23a.sf100/explain.txt | 8 +- .../q23a.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q23a/explain.txt | 8 +- .../approved-plans-v1_4/q23a/simplified.txt | 4 +- .../q23b.sf100/explain.txt | 8 +- .../q23b.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q23b/explain.txt | 8 +- .../approved-plans-v1_4/q23b/simplified.txt | 4 +- .../approved-plans-v1_4/q33.sf100/explain.txt | 8 +- .../q33.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q33/explain.txt | 8 +- .../approved-plans-v1_4/q33/simplified.txt | 4 +- .../approved-plans-v1_4/q35.sf100/explain.txt | 8 +- .../q35.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q35/explain.txt | 8 +- .../approved-plans-v1_4/q35/simplified.txt | 4 +- .../approved-plans-v1_4/q5.sf100/explain.txt | 11 +- .../q5.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q5/explain.txt | 11 +- .../approved-plans-v1_4/q5/simplified.txt | 2 +- .../approved-plans-v1_4/q56.sf100/explain.txt | 8 +- .../q56.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q56/explain.txt | 8 +- .../approved-plans-v1_4/q56/simplified.txt | 4 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 24 +- .../q58.sf100/simplified.txt | 12 +- .../approved-plans-v1_4/q58/explain.txt | 24 +- .../approved-plans-v1_4/q58/simplified.txt | 12 +- .../approved-plans-v1_4/q60.sf100/explain.txt | 8 +- .../q60.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q60/explain.txt | 8 +- .../approved-plans-v1_4/q60/simplified.txt | 4 +- .../approved-plans-v1_4/q69.sf100/explain.txt | 16 +- .../q69.sf100/simplified.txt | 8 +- .../approved-plans-v1_4/q69/explain.txt | 16 +- .../approved-plans-v1_4/q69/simplified.txt | 8 +- .../approved-plans-v1_4/q70.sf100/explain.txt | 6 +- .../q70.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q70/explain.txt | 6 +- .../approved-plans-v1_4/q70/simplified.txt | 4 +- .../approved-plans-v1_4/q83.sf100/explain.txt | 229 +++++++++--------- .../q83.sf100/simplified.txt | 11 +- .../approved-plans-v1_4/q83/explain.txt | 205 ++++++++-------- .../approved-plans-v1_4/q83/simplified.txt | 11 +- .../approved-plans-v1_4/q93.sf100/explain.txt | 42 ++-- .../q93.sf100/simplified.txt | 7 +- .../approved-plans-v1_4/q93/explain.txt | 78 +++--- .../approved-plans-v1_4/q93/simplified.txt | 7 +- .../approved-plans-v1_4/q94.sf100/explain.txt | 180 +++++++------- .../q94.sf100/simplified.txt | 16 +- .../approved-plans-v1_4/q94/explain.txt | 166 +++++++------ .../approved-plans-v1_4/q94/simplified.txt | 16 +- .../approved-plans-v1_4/q95.sf100/explain.txt | 4 +- .../q95.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q95/explain.txt | 4 +- .../approved-plans-v1_4/q95/simplified.txt | 2 +- .../q10a.sf100/explain.txt | 16 +- .../q10a.sf100/simplified.txt | 8 +- .../approved-plans-v2_7/q10a/explain.txt | 16 +- .../approved-plans-v2_7/q10a/simplified.txt | 8 +- .../approved-plans-v2_7/q14.sf100/explain.txt | 4 +- .../q14.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q14/explain.txt | 4 +- .../approved-plans-v2_7/q14/simplified.txt | 2 +- .../q14a.sf100/explain.txt | 4 +- .../q14a.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q14a/explain.txt | 4 +- .../approved-plans-v2_7/q14a/simplified.txt | 2 +- .../approved-plans-v2_7/q35.sf100/explain.txt | 8 +- .../q35.sf100/simplified.txt | 4 +- .../approved-plans-v2_7/q35/explain.txt | 8 +- .../approved-plans-v2_7/q35/simplified.txt | 4 +- .../q35a.sf100/explain.txt | 16 +- .../q35a.sf100/simplified.txt | 8 +- .../approved-plans-v2_7/q35a/explain.txt | 16 +- .../approved-plans-v2_7/q35a/simplified.txt | 8 +- .../approved-plans-v2_7/q5a.sf100/explain.txt | 11 +- .../q5a.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q5a/explain.txt | 11 +- .../approved-plans-v2_7/q5a/simplified.txt | 2 +- .../q70a.sf100/explain.txt | 6 +- .../q70a.sf100/simplified.txt | 4 +- .../approved-plans-v2_7/q70a/explain.txt | 6 +- .../approved-plans-v2_7/q70a/simplified.txt | 4 +- 100 files changed, 992 insertions(+), 902 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt index 72f9339134e87..4eb2ac8a46901 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt @@ -63,7 +63,7 @@ TakeOrderedAndProject (58) Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -71,7 +71,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -85,7 +85,7 @@ Arguments: [c_customer_sk#3 ASC NULLS FIRST], false, 0 Output [2]: [ss_sold_date_sk#7, ss_customer_sk#8] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] @@ -93,7 +93,7 @@ Input [2]: [ss_sold_date_sk#7, ss_customer_sk#8] (8) Filter [codegen id : 4] Input [2]: [ss_sold_date_sk#7, ss_customer_sk#8] -Condition : isnotnull(ss_sold_date_sk#7) +Condition : (isnotnull(ss_sold_date_sk#7) AND isnotnull(ss_customer_sk#8)) (9) Scan parquet default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt index 32dfbc7173529..3f8a5aaf4b479 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt @@ -26,7 +26,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] @@ -37,7 +37,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index f9e871077f684..bf520b4384c61 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -55,7 +55,7 @@ TakeOrderedAndProject (50) Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -63,13 +63,13 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#6, ss_customer_sk#7] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] @@ -77,7 +77,7 @@ Input [2]: [ss_sold_date_sk#6, ss_customer_sk#7] (6) Filter [codegen id : 2] Input [2]: [ss_sold_date_sk#6, ss_customer_sk#7] -Condition : isnotnull(ss_sold_date_sk#6) +Condition : (isnotnull(ss_sold_date_sk#6) AND isnotnull(ss_customer_sk#7)) (7) Scan parquet default.date_dim Output [3]: [d_date_sk#8, d_year#9, d_moy#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 19781433465a3..078b3501b35ea 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] @@ -23,7 +23,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] 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 c03db659679ce..86883d6f4d65c 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 @@ -158,7 +158,7 @@ Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/item] -PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_brand_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 17] @@ -166,7 +166,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (8) 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_class_id#8) AND isnotnull(i_brand_id#7)) AND isnotnull(i_category_id#9)) +Condition : (((isnotnull(i_class_id#8) AND isnotnull(i_brand_id#7)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (9) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] 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 71d4c167f6564..e5f7678bfa641 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 @@ -80,7 +80,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,sum(number_sa WholeStageCodegen (17) Project [i_item_sk] BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index c96b1c502a15f..ec35d6d258db0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -134,7 +134,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/item] -PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_brand_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -142,7 +142,7 @@ Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : ((isnotnull(i_class_id#7) AND isnotnull(i_brand_id#6)) AND isnotnull(i_category_id#8)) +Condition : (((isnotnull(i_class_id#7) AND isnotnull(i_brand_id#6)) AND isnotnull(i_category_id#8)) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index 9facf93115f1f..959237dcc8657 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,sum(number_sa WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] 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 8c27f7ddce018..178e1e3870180 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 @@ -137,7 +137,7 @@ Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/item] -PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_brand_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 17] @@ -145,7 +145,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (8) 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_class_id#8) AND isnotnull(i_brand_id#7)) AND isnotnull(i_category_id#9)) +Condition : (((isnotnull(i_class_id#8) AND isnotnull(i_brand_id#7)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (9) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] 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 5141d40e7325b..ee77670ddc6b3 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 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat WholeStageCodegen (17) Project [i_item_sk] BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 54e984da09306..23282923db00b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -119,7 +119,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/item] -PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_brand_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -127,7 +127,7 @@ Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : ((isnotnull(i_class_id#7) AND isnotnull(i_brand_id#6)) AND isnotnull(i_category_id#8)) +Condition : (((isnotnull(i_class_id#7) AND isnotnull(i_brand_id#6)) AND isnotnull(i_category_id#8)) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index f1668ea399807..7892b6727f9a3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index 66bf2dc518751..c2697fda2f1a1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -1,55 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * HashAggregate (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- SortMergeJoin LeftAnti (17) - : : : :- * Project (12) - : : : : +- SortMergeJoin LeftSemi (11) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- SortMergeJoin LeftSemi (12) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.catalog_sales (1) - : : : : +- * Sort (10) - : : : : +- Exchange (9) - : : : : +- * Project (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.catalog_sales (6) - : : : +- * Sort (16) - : : : +- Exchange (15) - : : : +- * ColumnarToRow (14) - : : : +- Scan parquet default.catalog_returns (13) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- * Filter (20) - : : +- * ColumnarToRow (19) - : : +- Scan parquet default.customer_address (18) - : +- BroadcastExchange (29) - : +- * Project (28) - : +- * Filter (27) - : +- * ColumnarToRow (26) - : +- Scan parquet default.call_center (25) - +- BroadcastExchange (36) - +- * Project (35) - +- * Filter (34) - +- * ColumnarToRow (33) - +- Scan parquet default.date_dim (32) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * Project (9) + : : : : +- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.catalog_sales (6) + : : : +- * Sort (18) + : : : +- Exchange (17) + : : : +- * Filter (16) + : : : +- * ColumnarToRow (15) + : : : +- Scan parquet default.catalog_returns (14) + : : +- BroadcastExchange (24) + : : +- * Project (23) + : : +- * Filter (22) + : : +- * ColumnarToRow (21) + : : +- Scan parquet default.customer_address (20) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- * ColumnarToRow (28) + : +- Scan parquet default.call_center (27) + +- BroadcastExchange (38) + +- * Project (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.date_dim (34) (1) Scan parquet default.catalog_sales Output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] +PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -57,7 +59,7 @@ Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho (3) Filter [codegen id : 1] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) +Condition : ((((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_order_number#5)) AND isnotnull(cs_warehouse_sk#4)) (4) Exchange Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -71,180 +73,190 @@ Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 Output [2]: [cs_warehouse_sk#4, cs_order_number#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/catalog_sales] +PushedFilters: [IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 3] Input [2]: [cs_warehouse_sk#4, cs_order_number#5] -(8) Project [codegen id : 3] +(8) Filter [codegen id : 3] +Input [2]: [cs_warehouse_sk#4, cs_order_number#5] +Condition : (isnotnull(cs_order_number#5) AND isnotnull(cs_warehouse_sk#4)) + +(9) Project [codegen id : 3] Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#9, cs_order_number#5 AS cs_order_number#5#10] Input [2]: [cs_warehouse_sk#4, cs_order_number#5] -(9) Exchange +(10) Exchange Input [2]: [cs_warehouse_sk#4#9, cs_order_number#5#10] Arguments: hashpartitioning(cs_order_number#5#10, 5), true, [id=#11] -(10) Sort [codegen id : 4] +(11) Sort [codegen id : 4] Input [2]: [cs_warehouse_sk#4#9, cs_order_number#5#10] Arguments: [cs_order_number#5#10 ASC NULLS FIRST], false, 0 -(11) SortMergeJoin +(12) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#5#10] Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#9) -(12) Project [codegen id : 5] +(13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(13) Scan parquet default.catalog_returns +(14) Scan parquet default.catalog_returns Output [1]: [cr_order_number#12] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/catalog_returns] +PushedFilters: [IsNotNull(cr_order_number)] ReadSchema: struct -(14) ColumnarToRow [codegen id : 6] +(15) ColumnarToRow [codegen id : 6] +Input [1]: [cr_order_number#12] + +(16) Filter [codegen id : 6] Input [1]: [cr_order_number#12] +Condition : isnotnull(cr_order_number#12) -(15) Exchange +(17) Exchange Input [1]: [cr_order_number#12] Arguments: hashpartitioning(cr_order_number#12, 5), true, [id=#13] -(16) Sort [codegen id : 7] +(18) Sort [codegen id : 7] Input [1]: [cr_order_number#12] Arguments: [cr_order_number#12 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin +(19) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#12] Join condition: None -(18) Scan parquet default.customer_address +(20) Scan parquet default.customer_address Output [2]: [ca_address_sk#14, ca_state#15] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 8] +(21) ColumnarToRow [codegen id : 8] Input [2]: [ca_address_sk#14, ca_state#15] -(20) Filter [codegen id : 8] +(22) Filter [codegen id : 8] Input [2]: [ca_address_sk#14, ca_state#15] Condition : ((isnotnull(ca_state#15) AND (ca_state#15 = GA)) AND isnotnull(ca_address_sk#14)) -(21) Project [codegen id : 8] +(23) Project [codegen id : 8] Output [1]: [ca_address_sk#14] Input [2]: [ca_address_sk#14, ca_state#15] -(22) BroadcastExchange +(24) BroadcastExchange Input [1]: [ca_address_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(23) BroadcastHashJoin [codegen id : 11] +(25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#14] Join condition: None -(24) Project [codegen id : 11] +(26) Project [codegen id : 11] Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#14] -(25) Scan parquet default.call_center +(27) Scan parquet default.call_center Output [2]: [cc_call_center_sk#17, cc_county#18] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 9] +(28) ColumnarToRow [codegen id : 9] Input [2]: [cc_call_center_sk#17, cc_county#18] -(27) Filter [codegen id : 9] +(29) Filter [codegen id : 9] Input [2]: [cc_call_center_sk#17, cc_county#18] Condition : ((isnotnull(cc_county#18) AND (cc_county#18 = Williamson County)) AND isnotnull(cc_call_center_sk#17)) -(28) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [1]: [cc_call_center_sk#17] Input [2]: [cc_call_center_sk#17, cc_county#18] -(29) BroadcastExchange +(31) BroadcastExchange Input [1]: [cc_call_center_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(30) BroadcastHashJoin [codegen id : 11] +(32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#17] Join condition: None -(31) Project [codegen id : 11] +(33) Project [codegen id : 11] Output [4]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#17] -(32) Scan parquet default.date_dim +(34) Scan parquet default.date_dim Output [2]: [d_date_sk#20, d_date#21] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 10] +(35) ColumnarToRow [codegen id : 10] Input [2]: [d_date_sk#20, d_date#21] -(34) Filter [codegen id : 10] +(36) Filter [codegen id : 10] Input [2]: [d_date_sk#20, d_date#21] Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 11719)) AND (d_date#21 <= 11779)) AND isnotnull(d_date_sk#20)) -(35) Project [codegen id : 10] +(37) Project [codegen id : 10] Output [1]: [d_date_sk#20] Input [2]: [d_date_sk#20, d_date#21] -(36) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(37) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#20] Join condition: None -(38) Project [codegen id : 11] +(40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#20] -(39) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#23, sum(UnscaledValue(cs_net_profit#7))#24] Results [3]: [cs_order_number#5, sum#25, sum#26] -(40) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#25, sum#26] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#23, sum(UnscaledValue(cs_net_profit#7))#24] Results [3]: [cs_order_number#5, sum#25, sum#26] -(41) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#25, sum#26] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#23, sum(UnscaledValue(cs_net_profit#7))#24, count(cs_order_number#5)#27] Results [3]: [sum#25, sum#26, count#28] -(42) Exchange +(44) Exchange Input [3]: [sum#25, sum#26, count#28] Arguments: SinglePartition, true, [id=#29] -(43) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 12] Input [3]: [sum#25, sum#26, count#28] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#23, sum(UnscaledValue(cs_net_profit#7))#24, count(cs_order_number#5)#27] Results [3]: [count(cs_order_number#5)#27 AS order count #30, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#23,17,2) AS total shipping cost #31, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#24,17,2) AS total net profit #32] -(44) TakeOrderedAndProject +(46) TakeOrderedAndProject Input [3]: [order count #30, total shipping cost #31, total net profit #32] Arguments: 100, [order count #30 ASC NULLS FIRST], [order count #30, total shipping cost #31, total net profit #32] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt index dc78021b94fbf..cd1e095f6c334 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] InputAdapter Exchange [cs_order_number] #2 WholeStageCodegen (1) - Filter [cs_call_center_sk,cs_ship_addr_sk,cs_ship_date_sk] + Filter [cs_call_center_sk,cs_order_number,cs_ship_addr_sk,cs_ship_date_sk,cs_warehouse_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_call_center_sk,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_ship_addr_sk,cs_ship_date_sk,cs_warehouse_sk] @@ -34,17 +34,19 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Exchange [cs_order_number] #3 WholeStageCodegen (3) Project [cs_order_number,cs_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_order_number,cs_warehouse_sk] + Filter [cs_order_number,cs_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_order_number,cs_warehouse_sk] WholeStageCodegen (7) Sort [cr_order_number] InputAdapter Exchange [cr_order_number] #4 WholeStageCodegen (6) - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_order_number] + Filter [cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_order_number] InputAdapter BroadcastExchange #5 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index ed45f7de91759..a3d40fd509658 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -1,52 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * HashAggregate (40) - +- Exchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * BroadcastHashJoin LeftAnti BuildRight (13) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (8) +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (29) + : +- * BroadcastHashJoin Inner BuildRight (28) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * BroadcastHashJoin LeftAnti BuildRight (15) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (9) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.catalog_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Project (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.catalog_sales (4) - : : : +- BroadcastExchange (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.catalog_returns (10) - : : +- BroadcastExchange (18) - : : +- * Project (17) - : : +- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet default.date_dim (14) - : +- BroadcastExchange (25) - : +- * Project (24) - : +- * Filter (23) - : +- * ColumnarToRow (22) - : +- Scan parquet default.customer_address (21) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- * ColumnarToRow (29) - +- Scan parquet default.call_center (28) + : : : : +- BroadcastExchange (8) + : : : : +- * Project (7) + : : : : +- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.catalog_sales (4) + : : : +- BroadcastExchange (14) + : : : +- * Filter (13) + : : : +- * ColumnarToRow (12) + : : : +- Scan parquet default.catalog_returns (11) + : : +- BroadcastExchange (20) + : : +- * Project (19) + : : +- * Filter (18) + : : +- * ColumnarToRow (17) + : : +- Scan parquet default.date_dim (16) + : +- BroadcastExchange (27) + : +- * Project (26) + : +- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.customer_address (23) + +- BroadcastExchange (34) + +- * Project (33) + +- * Filter (32) + +- * ColumnarToRow (31) + +- Scan parquet default.call_center (30) (1) Scan parquet default.catalog_sales Output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] +PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 6] @@ -54,182 +56,192 @@ Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho (3) Filter [codegen id : 6] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) +Condition : ((((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_order_number#5)) AND isnotnull(cs_warehouse_sk#4)) (4) Scan parquet default.catalog_sales Output [2]: [cs_warehouse_sk#4, cs_order_number#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/catalog_sales] +PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_order_number)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] Input [2]: [cs_warehouse_sk#4, cs_order_number#5] -(6) Project [codegen id : 1] +(6) Filter [codegen id : 1] +Input [2]: [cs_warehouse_sk#4, cs_order_number#5] +Condition : (isnotnull(cs_warehouse_sk#4) AND isnotnull(cs_order_number#5)) + +(7) Project [codegen id : 1] Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#8, cs_order_number#5 AS cs_order_number#5#9] Input [2]: [cs_warehouse_sk#4, cs_order_number#5] -(7) BroadcastExchange +(8) BroadcastExchange Input [2]: [cs_warehouse_sk#4#8, cs_order_number#5#9] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#10] -(8) BroadcastHashJoin [codegen id : 6] +(9) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#5#9] Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#8) -(9) Project [codegen id : 6] +(10) Project [codegen id : 6] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(10) Scan parquet default.catalog_returns +(11) Scan parquet default.catalog_returns Output [1]: [cr_order_number#11] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/catalog_returns] +PushedFilters: [IsNotNull(cr_order_number)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 2] +(12) ColumnarToRow [codegen id : 2] +Input [1]: [cr_order_number#11] + +(13) Filter [codegen id : 2] Input [1]: [cr_order_number#11] +Condition : isnotnull(cr_order_number#11) -(12) BroadcastExchange +(14) BroadcastExchange Input [1]: [cr_order_number#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] -(13) BroadcastHashJoin [codegen id : 6] +(15) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#11] Join condition: None -(14) Scan parquet default.date_dim +(16) Scan parquet default.date_dim Output [2]: [d_date_sk#13, d_date#14] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 3] +(17) ColumnarToRow [codegen id : 3] Input [2]: [d_date_sk#13, d_date#14] -(16) Filter [codegen id : 3] +(18) Filter [codegen id : 3] Input [2]: [d_date_sk#13, d_date#14] Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 11719)) AND (d_date#14 <= 11779)) AND isnotnull(d_date_sk#13)) -(17) Project [codegen id : 3] +(19) Project [codegen id : 3] Output [1]: [d_date_sk#13] Input [2]: [d_date_sk#13, d_date#14] -(18) BroadcastExchange +(20) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] -(19) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#13] Join condition: None -(20) Project [codegen id : 6] +(22) Project [codegen id : 6] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#13] -(21) Scan parquet default.customer_address +(23) Scan parquet default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(22) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 4] Input [2]: [ca_address_sk#16, ca_state#17] -(23) Filter [codegen id : 4] +(25) Filter [codegen id : 4] Input [2]: [ca_address_sk#16, ca_state#17] Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_address_sk#16)) -(24) Project [codegen id : 4] +(26) Project [codegen id : 4] Output [1]: [ca_address_sk#16] Input [2]: [ca_address_sk#16, ca_state#17] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(26) BroadcastHashJoin [codegen id : 6] +(28) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join condition: None -(27) Project [codegen id : 6] +(29) Project [codegen id : 6] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -(28) Scan parquet default.call_center +(30) Scan parquet default.call_center Output [2]: [cc_call_center_sk#19, cc_county#20] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 5] +(31) ColumnarToRow [codegen id : 5] Input [2]: [cc_call_center_sk#19, cc_county#20] -(30) Filter [codegen id : 5] +(32) Filter [codegen id : 5] Input [2]: [cc_call_center_sk#19, cc_county#20] Condition : ((isnotnull(cc_county#20) AND (cc_county#20 = Williamson County)) AND isnotnull(cc_call_center_sk#19)) -(31) Project [codegen id : 5] +(33) Project [codegen id : 5] Output [1]: [cc_call_center_sk#19] Input [2]: [cc_call_center_sk#19, cc_county#20] -(32) BroadcastExchange +(34) BroadcastExchange Input [1]: [cc_call_center_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(33) BroadcastHashJoin [codegen id : 6] +(35) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#19] Join condition: None -(34) Project [codegen id : 6] +(36) Project [codegen id : 6] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#19] -(35) HashAggregate [codegen id : 6] +(37) HashAggregate [codegen id : 6] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23] Results [3]: [cs_order_number#5, sum#24, sum#25] -(36) Exchange +(38) Exchange Input [3]: [cs_order_number#5, sum#24, sum#25] Arguments: hashpartitioning(cs_order_number#5, 5), true, [id=#26] -(37) HashAggregate [codegen id : 7] +(39) HashAggregate [codegen id : 7] Input [3]: [cs_order_number#5, sum#24, sum#25] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23] Results [3]: [cs_order_number#5, sum#24, sum#25] -(38) HashAggregate [codegen id : 7] +(40) HashAggregate [codegen id : 7] Input [3]: [cs_order_number#5, sum#24, sum#25] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#27] Results [3]: [sum#24, sum#25, count#28] -(39) Exchange +(41) Exchange Input [3]: [sum#24, sum#25, count#28] Arguments: SinglePartition, true, [id=#29] -(40) HashAggregate [codegen id : 8] +(42) HashAggregate [codegen id : 8] Input [3]: [sum#24, sum#25, count#28] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#27] Results [3]: [count(cs_order_number#5)#27 AS order count #30, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#22,17,2) AS total shipping cost #31, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#23,17,2) AS total net profit #32] -(41) TakeOrderedAndProject +(43) TakeOrderedAndProject Input [3]: [order count #30, total shipping cost #31, total net profit #32] Arguments: 100, [order count #30 ASC NULLS FIRST], [order count #30, total shipping cost #31, total net profit #32] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index cdf86a4813208..d90c0c1a7ab65 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] BroadcastHashJoin [cr_order_number,cs_order_number] Project [cs_call_center_sk,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_ship_addr_sk,cs_ship_date_sk] BroadcastHashJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] - Filter [cs_call_center_sk,cs_ship_addr_sk,cs_ship_date_sk] + Filter [cs_call_center_sk,cs_order_number,cs_ship_addr_sk,cs_ship_date_sk,cs_warehouse_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_call_center_sk,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_ship_addr_sk,cs_ship_date_sk,cs_warehouse_sk] @@ -27,15 +27,17 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] BroadcastExchange #3 WholeStageCodegen (1) Project [cs_order_number,cs_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_order_number,cs_warehouse_sk] + Filter [cs_order_number,cs_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_order_number,cs_warehouse_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_order_number] + Filter [cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_order_number] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt index 702901b354289..782beb84365c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt @@ -97,7 +97,7 @@ CollectLimit (92) Output [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -105,7 +105,7 @@ Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity# (3) Filter [codegen id : 1] Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] -Condition : isnotnull(cs_sold_date_sk#1) +Condition : ((isnotnull(cs_sold_date_sk#1) AND isnotnull(cs_item_sk#3)) AND isnotnull(cs_bill_customer_sk#2)) (4) Exchange Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] @@ -369,7 +369,7 @@ Input [4]: [cs_sold_date_sk#1, cs_quantity#4, cs_list_price#5, d_date_sk#9] Output [5]: [ws_sold_date_sk#42, ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 19] @@ -377,7 +377,7 @@ Input [5]: [ws_sold_date_sk#42, ws_item_sk#43, ws_bill_customer_sk#44, ws_quanti (63) Filter [codegen id : 19] Input [5]: [ws_sold_date_sk#42, ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] -Condition : isnotnull(ws_sold_date_sk#42) +Condition : ((isnotnull(ws_sold_date_sk#42) AND isnotnull(ws_item_sk#43)) AND isnotnull(ws_bill_customer_sk#44)) (64) Exchange Input [5]: [ws_sold_date_sk#42, ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt index 7b08c6a571b4c..01f77b0292853 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt @@ -26,7 +26,7 @@ CollectLimit InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (1) - Filter [cs_sold_date_sk] + Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_list_price,cs_quantity,cs_sold_date_sk] @@ -167,7 +167,7 @@ CollectLimit InputAdapter Exchange [ws_item_sk] #16 WholeStageCodegen (19) - Filter [ws_sold_date_sk] + Filter [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_list_price,ws_quantity,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index 37c10d8acd77b..7cf46ac5180d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -77,7 +77,7 @@ CollectLimit (72) Output [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -85,7 +85,7 @@ Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity# (3) Filter [codegen id : 9] Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] -Condition : isnotnull(cs_sold_date_sk#1) +Condition : ((isnotnull(cs_sold_date_sk#1) AND isnotnull(cs_item_sk#3)) AND isnotnull(cs_bill_customer_sk#2)) (4) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#6, ss_item_sk#7] @@ -313,7 +313,7 @@ Input [4]: [cs_sold_date_sk#1, cs_quantity#4, cs_list_price#5, d_date_sk#8] Output [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (53) ColumnarToRow [codegen id : 18] @@ -321,7 +321,7 @@ Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quanti (54) Filter [codegen id : 18] Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] -Condition : isnotnull(ws_sold_date_sk#41) +Condition : ((isnotnull(ws_sold_date_sk#41) AND isnotnull(ws_item_sk#42)) AND isnotnull(ws_bill_customer_sk#43)) (55) ReusedExchange [Reuses operator id: 25] Output [1]: [item_sk#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index a69293edd6218..bb065c04387ae 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -14,7 +14,7 @@ CollectLimit BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] Project [cs_bill_customer_sk,cs_list_price,cs_quantity,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_sold_date_sk] + Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_list_price,cs_quantity,cs_sold_date_sk] @@ -124,7 +124,7 @@ CollectLimit BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] Project [ws_bill_customer_sk,ws_list_price,ws_quantity,ws_sold_date_sk] BroadcastHashJoin [item_sk,ws_item_sk] - Filter [ws_sold_date_sk] + Filter [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_list_price,ws_quantity,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt index 6039d13e74edb..1219e3a6aa60d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt @@ -135,7 +135,7 @@ TakeOrderedAndProject (130) Output [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -143,7 +143,7 @@ Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity# (3) Filter [codegen id : 1] Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] -Condition : (isnotnull(cs_bill_customer_sk#2) AND isnotnull(cs_sold_date_sk#1)) +Condition : ((isnotnull(cs_bill_customer_sk#2) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#3)) (4) Exchange Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] @@ -510,7 +510,7 @@ Results [3]: [c_last_name#42, c_first_name#41, sum(CheckOverflow((promote_precis Output [5]: [ws_sold_date_sk#53, ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/web_sales] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk)] +PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 29] @@ -518,7 +518,7 @@ Input [5]: [ws_sold_date_sk#53, ws_item_sk#54, ws_bill_customer_sk#55, ws_quanti (85) Filter [codegen id : 29] Input [5]: [ws_sold_date_sk#53, ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57] -Condition : (isnotnull(ws_bill_customer_sk#55) AND isnotnull(ws_sold_date_sk#53)) +Condition : ((isnotnull(ws_bill_customer_sk#55) AND isnotnull(ws_sold_date_sk#53)) AND isnotnull(ws_item_sk#54)) (86) Exchange Input [5]: [ws_sold_date_sk#53, ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt index 3964f50895888..79999393ad374 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt @@ -27,7 +27,7 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (1) - Filter [cs_bill_customer_sk,cs_sold_date_sk] + Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_list_price,cs_quantity,cs_sold_date_sk] @@ -209,7 +209,7 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] InputAdapter Exchange [ws_item_sk] #19 WholeStageCodegen (29) - Filter [ws_bill_customer_sk,ws_sold_date_sk] + Filter [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_list_price,ws_quantity,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 61e4b21189a86..56153683fc883 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -102,7 +102,7 @@ TakeOrderedAndProject (97) Output [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 13] @@ -110,7 +110,7 @@ Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity# (3) Filter [codegen id : 13] Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] -Condition : (isnotnull(cs_bill_customer_sk#2) AND isnotnull(cs_sold_date_sk#1)) +Condition : ((isnotnull(cs_bill_customer_sk#2) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#3)) (4) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#6, ss_item_sk#7] @@ -406,7 +406,7 @@ Results [3]: [c_last_name#39, c_first_name#38, sum(CheckOverflow((promote_precis Output [5]: [ws_sold_date_sk#52, ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/web_sales] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk)] +PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 27] @@ -414,7 +414,7 @@ Input [5]: [ws_sold_date_sk#52, ws_item_sk#53, ws_bill_customer_sk#54, ws_quanti (68) Filter [codegen id : 27] Input [5]: [ws_sold_date_sk#52, ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56] -Condition : (isnotnull(ws_bill_customer_sk#54) AND isnotnull(ws_sold_date_sk#52)) +Condition : ((isnotnull(ws_bill_customer_sk#54) AND isnotnull(ws_sold_date_sk#52)) AND isnotnull(ws_item_sk#53)) (69) ReusedExchange [Reuses operator id: 25] Output [1]: [item_sk#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 277966b363866..c444ac3c443a8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] Project [cs_bill_customer_sk,cs_list_price,cs_quantity,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_bill_customer_sk,cs_sold_date_sk] + Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_list_price,cs_quantity,cs_sold_date_sk] @@ -146,7 +146,7 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] Project [ws_bill_customer_sk,ws_list_price,ws_quantity,ws_sold_date_sk] BroadcastHashJoin [item_sk,ws_item_sk] - Filter [ws_bill_customer_sk,ws_sold_date_sk] + Filter [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_list_price,ws_quantity,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index 044f552befdd8..dacc93848b277 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -117,7 +117,7 @@ Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, Output [2]: [i_item_sk#9, i_manufact_id#10] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_manufact_id)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] @@ -125,13 +125,13 @@ Input [2]: [i_item_sk#9, i_manufact_id#10] (13) Filter [codegen id : 3] Input [2]: [i_item_sk#9, i_manufact_id#10] -Condition : isnotnull(i_item_sk#9) +Condition : (isnotnull(i_item_sk#9) AND isnotnull(i_manufact_id#10)) (14) Scan parquet default.item Output [2]: [i_category#11, i_manufact_id#10] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics), IsNotNull(i_manufact_id)] ReadSchema: struct (15) ColumnarToRow [codegen id : 2] @@ -139,7 +139,7 @@ Input [2]: [i_category#11, i_manufact_id#10] (16) Filter [codegen id : 2] Input [2]: [i_category#11, i_manufact_id#10] -Condition : (isnotnull(i_category#11) AND (i_category#11 = Electronics)) +Condition : ((isnotnull(i_category#11) AND (i_category#11 = Electronics)) AND isnotnull(i_manufact_id#10)) (17) Project [codegen id : 2] Output [1]: [i_manufact_id#10 AS i_manufact_id#10#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index e810c4bfbf62d..24d4255b9a5ef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -35,7 +35,7 @@ TakeOrderedAndProject [i_manufact_id,total_sales] BroadcastExchange #4 WholeStageCodegen (3) BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] + Filter [i_item_sk,i_manufact_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] @@ -43,7 +43,7 @@ TakeOrderedAndProject [i_manufact_id,total_sales] BroadcastExchange #5 WholeStageCodegen (2) Project [i_manufact_id] - Filter [i_category] + Filter [i_category,i_manufact_id] ColumnarToRow InputAdapter Scan parquet default.item [i_category,i_manufact_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index 78227141b3a2a..dd98b7f1f0e2a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -148,7 +148,7 @@ Input [4]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, ca_address_sk#9] Output [2]: [i_item_sk#12, i_manufact_id#13] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_manufact_id)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -156,13 +156,13 @@ Input [2]: [i_item_sk#12, i_manufact_id#13] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#12, i_manufact_id#13] -Condition : isnotnull(i_item_sk#12) +Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_manufact_id#13)) (21) Scan parquet default.item Output [2]: [i_category#14, i_manufact_id#13] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics), IsNotNull(i_manufact_id)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -170,7 +170,7 @@ Input [2]: [i_category#14, i_manufact_id#13] (23) Filter [codegen id : 3] Input [2]: [i_category#14, i_manufact_id#13] -Condition : (isnotnull(i_category#14) AND (i_category#14 = Electronics)) +Condition : ((isnotnull(i_category#14) AND (i_category#14 = Electronics)) AND isnotnull(i_manufact_id#13)) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#13 AS i_manufact_id#13#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index 5b0cca34b3c70..8bbcd24fc243b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject [i_manufact_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] + Filter [i_item_sk,i_manufact_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] @@ -51,7 +51,7 @@ TakeOrderedAndProject [i_manufact_id,total_sales] BroadcastExchange #6 WholeStageCodegen (3) Project [i_manufact_id] - Filter [i_category] + Filter [i_category,i_manufact_id] ColumnarToRow InputAdapter Scan parquet default.item [i_category,i_manufact_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index 527e77d7a7afc..3849271121687 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -65,7 +65,7 @@ TakeOrderedAndProject (60) Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -73,7 +73,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -87,7 +87,7 @@ Arguments: [c_customer_sk#3 ASC NULLS FIRST], false, 0 Output [2]: [ss_sold_date_sk#7, ss_customer_sk#8] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] @@ -95,7 +95,7 @@ Input [2]: [ss_sold_date_sk#7, ss_customer_sk#8] (8) Filter [codegen id : 4] Input [2]: [ss_sold_date_sk#7, ss_customer_sk#8] -Condition : isnotnull(ss_sold_date_sk#7) +Condition : (isnotnull(ss_sold_date_sk#7) AND isnotnull(ss_customer_sk#8)) (9) Scan parquet default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt index 35dc74b2ced18..53fb2b797b2c5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [aggOrder,avg(cd_dep_college_count),avg(cd_dep_count),avg( InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] @@ -43,7 +43,7 @@ TakeOrderedAndProject [aggOrder,avg(cd_dep_college_count),avg(cd_dep_count),avg( WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index cb958fdb8abf4..0379bbaafe496 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -54,7 +54,7 @@ TakeOrderedAndProject (49) Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -62,13 +62,13 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#6, ss_customer_sk#7] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] @@ -76,7 +76,7 @@ Input [2]: [ss_sold_date_sk#6, ss_customer_sk#7] (6) Filter [codegen id : 2] Input [2]: [ss_sold_date_sk#6, ss_customer_sk#7] -Condition : isnotnull(ss_sold_date_sk#6) +Condition : (isnotnull(ss_sold_date_sk#6) AND isnotnull(ss_customer_sk#7)) (7) Scan parquet default.date_dim Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index 4f500e2b32b31..fb28975e221ab 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [aggOrder,avg(cd_dep_college_count),avg(cd_dep_count),avg( BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] @@ -23,7 +23,7 @@ TakeOrderedAndProject [aggOrder,avg(cd_dep_college_count),avg(cd_dep_count),avg( WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index 034301643add7..b6befb9b2df8f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -86,6 +86,7 @@ TakeOrderedAndProject (81) Output [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -93,7 +94,7 @@ Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profi (3) Filter [codegen id : 1] Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Condition : (isnotnull(cast(ss_sold_date_sk#1 as bigint)) AND isnotnull(cast(ss_store_sk#2 as bigint))) +Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#2)) AND isnotnull(cast(ss_sold_date_sk#1 as bigint))) AND isnotnull(cast(ss_store_sk#2 as bigint))) (4) Project [codegen id : 1] Output [6]: [cast(ss_store_sk#2 as bigint) AS store_sk#5, cast(ss_sold_date_sk#1 as bigint) AS date_sk#6, ss_ext_sales_price#3 AS sales_price#7, ss_net_profit#4 AS profit#8, 0.00 AS return_amt#9, 0.00 AS net_loss#10] @@ -294,7 +295,7 @@ Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#51))#77,17,2) AS sales#8 Output [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/web_sales] -PushedFilters: [IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] @@ -302,7 +303,7 @@ Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net (49) Filter [codegen id : 13] Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] -Condition : (isnotnull(cast(ws_sold_date_sk#86 as bigint)) AND isnotnull(ws_web_site_sk#87)) +Condition : ((isnotnull(ws_sold_date_sk#86) AND isnotnull(cast(ws_sold_date_sk#86 as bigint))) AND isnotnull(ws_web_site_sk#87)) (50) Project [codegen id : 13] Output [6]: [ws_web_site_sk#87 AS wsr_web_site_sk#90, cast(ws_sold_date_sk#86 as bigint) AS date_sk#91, ws_ext_sales_price#88 AS sales_price#92, ws_net_profit#89 AS profit#93, 0.00 AS return_amt#94, 0.00 AS net_loss#95] @@ -312,7 +313,7 @@ Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net Output [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/web_returns] -PushedFilters: [IsNotNull(wr_returned_date_sk)] +PushedFilters: [IsNotNull(wr_returned_date_sk), IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] @@ -320,7 +321,7 @@ Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return (53) Filter [codegen id : 14] Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] -Condition : isnotnull(wr_returned_date_sk#96) +Condition : ((isnotnull(wr_returned_date_sk#96) AND isnotnull(wr_item_sk#97)) AND isnotnull(wr_order_number#98)) (54) Exchange Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt index 71e204433c6ef..c4d9339b11a15 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt @@ -107,7 +107,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] InputAdapter Exchange [wr_item_sk,wr_order_number] #8 WholeStageCodegen (14) - Filter [wr_returned_date_sk] + Filter [wr_item_sk,wr_order_number,wr_returned_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_net_loss,wr_order_number,wr_return_amt,wr_returned_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index cbe5ed4a5b6aa..14a2c35cb53f5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -83,6 +83,7 @@ TakeOrderedAndProject (78) Output [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -90,7 +91,7 @@ Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profi (3) Filter [codegen id : 1] Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Condition : (isnotnull(cast(ss_sold_date_sk#1 as bigint)) AND isnotnull(cast(ss_store_sk#2 as bigint))) +Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#2)) AND isnotnull(cast(ss_sold_date_sk#1 as bigint))) AND isnotnull(cast(ss_store_sk#2 as bigint))) (4) Project [codegen id : 1] Output [6]: [cast(ss_store_sk#2 as bigint) AS store_sk#5, cast(ss_sold_date_sk#1 as bigint) AS date_sk#6, ss_ext_sales_price#3 AS sales_price#7, ss_net_profit#4 AS profit#8, 0.00 AS return_amt#9, 0.00 AS net_loss#10] @@ -291,7 +292,7 @@ Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#51))#77,17,2) AS sales#8 Output [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/web_sales] -PushedFilters: [IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] @@ -299,7 +300,7 @@ Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net (49) Filter [codegen id : 13] Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] -Condition : (isnotnull(cast(ws_sold_date_sk#86 as bigint)) AND isnotnull(ws_web_site_sk#87)) +Condition : ((isnotnull(ws_sold_date_sk#86) AND isnotnull(cast(ws_sold_date_sk#86 as bigint))) AND isnotnull(ws_web_site_sk#87)) (50) Project [codegen id : 13] Output [6]: [ws_web_site_sk#87 AS wsr_web_site_sk#90, cast(ws_sold_date_sk#86 as bigint) AS date_sk#91, ws_ext_sales_price#88 AS sales_price#92, ws_net_profit#89 AS profit#93, 0.00 AS return_amt#94, 0.00 AS net_loss#95] @@ -309,7 +310,7 @@ Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net Output [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/web_returns] -PushedFilters: [IsNotNull(wr_returned_date_sk)] +PushedFilters: [IsNotNull(wr_returned_date_sk), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 15] @@ -317,7 +318,7 @@ Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return (53) Filter [codegen id : 15] Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] -Condition : isnotnull(wr_returned_date_sk#96) +Condition : ((isnotnull(wr_returned_date_sk#96) AND isnotnull(wr_order_number#98)) AND isnotnull(wr_item_sk#97)) (54) Scan parquet default.web_sales Output [3]: [ws_item_sk#101, ws_web_site_sk#87, ws_order_number#102] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index 0bb6f48db3e12..8d21b523f30b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -101,7 +101,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] WholeStageCodegen (15) Project [wr_net_loss,wr_return_amt,wr_returned_date_sk,ws_web_site_sk] BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Filter [wr_returned_date_sk] + Filter [wr_item_sk,wr_order_number,wr_returned_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_net_loss,wr_order_number,wr_return_amt,wr_returned_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 480c3ecda13f3..77bdf99ad743a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -148,7 +148,7 @@ Input [4]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, ca_address_sk#9] Output [2]: [i_item_sk#12, i_item_id#13] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -156,13 +156,13 @@ Input [2]: [i_item_sk#12, i_item_id#13] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#12, i_item_id#13] -Condition : isnotnull(i_item_sk#12) +Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) (21) Scan parquet default.item Output [2]: [i_item_id#13, i_color#14] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/item] -PushedFilters: [In(i_color, [slate,blanched,burnished])] +PushedFilters: [In(i_color, [slate,blanched,burnished]), IsNotNull(i_item_id)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -170,7 +170,7 @@ Input [2]: [i_item_id#13, i_color#14] (23) Filter [codegen id : 3] Input [2]: [i_item_id#13, i_color#14] -Condition : i_color#14 IN (slate,blanched,burnished) +Condition : (i_color#14 IN (slate,blanched,burnished) AND isnotnull(i_item_id#13)) (24) Project [codegen id : 3] Output [1]: [i_item_id#13 AS i_item_id#13#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt index f8bc60fffb502..1da85841df199 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] + Filter [i_item_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_id,i_item_sk] @@ -51,7 +51,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_color] + Filter [i_color,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_color,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index ba02fc0bfcfd8..d860289a33ca6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -148,7 +148,7 @@ Input [4]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, ca_address_sk#9] Output [2]: [i_item_sk#12, i_item_id#13] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -156,13 +156,13 @@ Input [2]: [i_item_sk#12, i_item_id#13] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#12, i_item_id#13] -Condition : isnotnull(i_item_sk#12) +Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) (21) Scan parquet default.item Output [2]: [i_item_id#13, i_color#14] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/item] -PushedFilters: [In(i_color, [slate,blanched,burnished])] +PushedFilters: [In(i_color, [slate,blanched,burnished]), IsNotNull(i_item_id)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -170,7 +170,7 @@ Input [2]: [i_item_id#13, i_color#14] (23) Filter [codegen id : 3] Input [2]: [i_item_id#13, i_color#14] -Condition : i_color#14 IN (slate,blanched,burnished) +Condition : (i_color#14 IN (slate,blanched,burnished) AND isnotnull(i_item_id#13)) (24) Project [codegen id : 3] Output [1]: [i_item_id#13 AS i_item_id#13#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index f8bc60fffb502..1da85841df199 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] + Filter [i_item_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_id,i_item_sk] @@ -51,7 +51,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_color] + Filter [i_color,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_color,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index 467aa26dac4e5..582087c321946 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -98,7 +98,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) Output [2]: [d_date_sk#4, d_date#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] @@ -106,13 +106,13 @@ Input [2]: [d_date_sk#4, d_date#5] (6) Filter [codegen id : 2] Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_date#5)) (7) Scan parquet default.date_dim Output [2]: [d_date#5, d_week_seq#6] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] @@ -120,7 +120,7 @@ Input [2]: [d_date#5, d_week_seq#6] (9) Filter [codegen id : 1] Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = Subquery scalar-subquery#7, [id=#8])) +Condition : ((isnotnull(d_week_seq#6) AND (d_week_seq#6 = Subquery scalar-subquery#7, [id=#8])) AND isnotnull(d_date#5)) (10) Project [codegen id : 1] Output [1]: [d_date#5 AS d_date#5#9] @@ -219,7 +219,7 @@ Condition : (isnotnull(cs_item_sk#22) AND isnotnull(cs_sold_date_sk#21)) Output [2]: [d_date_sk#4, d_date#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (31) ColumnarToRow [codegen id : 6] @@ -227,13 +227,13 @@ Input [2]: [d_date_sk#4, d_date#5] (32) Filter [codegen id : 6] Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_date#5)) (33) Scan parquet default.date_dim Output [2]: [d_date#5, d_week_seq#6] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (34) ColumnarToRow [codegen id : 5] @@ -241,7 +241,7 @@ Input [2]: [d_date#5, d_week_seq#6] (35) Filter [codegen id : 5] Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) +Condition : ((isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) AND isnotnull(d_date#5)) (36) Project [codegen id : 5] Output [1]: [d_date#5 AS d_date#5#24] @@ -338,7 +338,7 @@ Condition : (isnotnull(ws_item_sk#35) AND isnotnull(ws_sold_date_sk#34)) Output [2]: [d_date_sk#4, d_date#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (57) ColumnarToRow [codegen id : 11] @@ -346,13 +346,13 @@ Input [2]: [d_date_sk#4, d_date#5] (58) Filter [codegen id : 11] Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_date#5)) (59) Scan parquet default.date_dim Output [2]: [d_date#5, d_week_seq#6] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (60) ColumnarToRow [codegen id : 10] @@ -360,7 +360,7 @@ Input [2]: [d_date#5, d_week_seq#6] (61) Filter [codegen id : 10] Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) +Condition : ((isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) AND isnotnull(d_date#5)) (62) Project [codegen id : 10] Output [1]: [d_date#5 AS d_date#5#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt index 0b846774d3c97..e62f5ca5f3d02 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ WholeStageCodegen (2) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_date_sk] @@ -31,7 +31,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ BroadcastExchange #3 WholeStageCodegen (1) Project [d_date] - Filter [d_week_seq] + Filter [d_date,d_week_seq] Subquery #1 WholeStageCodegen (1) Project [d_week_seq] @@ -71,7 +71,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ WholeStageCodegen (6) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_date_sk] @@ -79,7 +79,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ BroadcastExchange #8 WholeStageCodegen (5) Project [d_date] - Filter [d_week_seq] + Filter [d_date,d_week_seq] ReusedSubquery [d_week_seq] #1 ColumnarToRow InputAdapter @@ -108,7 +108,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ WholeStageCodegen (11) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_date_sk] @@ -116,7 +116,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ BroadcastExchange #12 WholeStageCodegen (10) Project [d_date] - Filter [d_week_seq] + Filter [d_date,d_week_seq] ReusedSubquery [d_week_seq] #1 ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index f587499d7d21a..6931c30bd77a1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -125,7 +125,7 @@ Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#4, Output [2]: [d_date_sk#7, d_date#8] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (11) ColumnarToRow [codegen id : 3] @@ -133,13 +133,13 @@ Input [2]: [d_date_sk#7, d_date#8] (12) Filter [codegen id : 3] Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Condition : (isnotnull(d_date_sk#7) AND isnotnull(d_date#8)) (13) Scan parquet default.date_dim Output [2]: [d_date#8, d_week_seq#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (14) ColumnarToRow [codegen id : 2] @@ -147,7 +147,7 @@ Input [2]: [d_date#8, d_week_seq#9] (15) Filter [codegen id : 2] Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = Subquery scalar-subquery#10, [id=#11])) +Condition : ((isnotnull(d_week_seq#9) AND (d_week_seq#9 = Subquery scalar-subquery#10, [id=#11])) AND isnotnull(d_date#8)) (16) Project [codegen id : 2] Output [1]: [d_date#8 AS d_date#8#12] @@ -231,7 +231,7 @@ Input [5]: [cs_sold_date_sk#21, cs_item_sk#22, cs_ext_sales_price#23, i_item_sk# Output [2]: [d_date_sk#7, d_date#8] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (34) ColumnarToRow [codegen id : 7] @@ -239,13 +239,13 @@ Input [2]: [d_date_sk#7, d_date#8] (35) Filter [codegen id : 7] Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Condition : (isnotnull(d_date_sk#7) AND isnotnull(d_date#8)) (36) Scan parquet default.date_dim Output [2]: [d_date#8, d_week_seq#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] @@ -253,7 +253,7 @@ Input [2]: [d_date#8, d_week_seq#9] (38) Filter [codegen id : 6] Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) +Condition : ((isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) AND isnotnull(d_date#8)) (39) Project [codegen id : 6] Output [1]: [d_date#8 AS d_date#8#24] @@ -350,7 +350,7 @@ Input [5]: [ws_sold_date_sk#34, ws_item_sk#35, ws_ext_sales_price#36, i_item_sk# Output [2]: [d_date_sk#7, d_date#8] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (60) ColumnarToRow [codegen id : 12] @@ -358,13 +358,13 @@ Input [2]: [d_date_sk#7, d_date#8] (61) Filter [codegen id : 12] Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Condition : (isnotnull(d_date_sk#7) AND isnotnull(d_date#8)) (62) Scan parquet default.date_dim Output [2]: [d_date#8, d_week_seq#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (63) ColumnarToRow [codegen id : 11] @@ -372,7 +372,7 @@ Input [2]: [d_date#8, d_week_seq#9] (64) Filter [codegen id : 11] Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) +Condition : ((isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) AND isnotnull(d_date#8)) (65) Project [codegen id : 11] Output [1]: [d_date#8 AS d_date#8#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 11295424fc7e3..7e9f64a684aca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -30,7 +30,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ WholeStageCodegen (3) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_date_sk] @@ -38,7 +38,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ BroadcastExchange #4 WholeStageCodegen (2) Project [d_date] - Filter [d_week_seq] + Filter [d_date,d_week_seq] Subquery #1 WholeStageCodegen (1) Project [d_week_seq] @@ -73,7 +73,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ WholeStageCodegen (7) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_date_sk] @@ -81,7 +81,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ BroadcastExchange #8 WholeStageCodegen (6) Project [d_date] - Filter [d_week_seq] + Filter [d_date,d_week_seq] ReusedSubquery [d_week_seq] #1 ColumnarToRow InputAdapter @@ -110,7 +110,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ WholeStageCodegen (12) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_date_sk] @@ -118,7 +118,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ BroadcastExchange #12 WholeStageCodegen (11) Project [d_date] - Filter [d_week_seq] + Filter [d_date,d_week_seq] ReusedSubquery [d_week_seq] #1 ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index cec4923c48198..df40e3e495cfb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -148,7 +148,7 @@ Input [4]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, ca_address_sk#9] Output [2]: [i_item_sk#12, i_item_id#13] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -156,13 +156,13 @@ Input [2]: [i_item_sk#12, i_item_id#13] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#12, i_item_id#13] -Condition : isnotnull(i_item_sk#12) +Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) (21) Scan parquet default.item Output [2]: [i_item_id#13, i_category#14] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music), IsNotNull(i_item_id)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -170,7 +170,7 @@ Input [2]: [i_item_id#13, i_category#14] (23) Filter [codegen id : 3] Input [2]: [i_item_id#13, i_category#14] -Condition : (isnotnull(i_category#14) AND (i_category#14 = Music)) +Condition : ((isnotnull(i_category#14) AND (i_category#14 = Music)) AND isnotnull(i_item_id#13)) (24) Project [codegen id : 3] Output [1]: [i_item_id#13 AS i_item_id#13#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt index d49bf8669bea5..3eb81f44edd45 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] + Filter [i_item_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_id,i_item_sk] @@ -51,7 +51,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_category] + Filter [i_category,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_category,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 8e05106d68911..8c88ce3b06bf3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -148,7 +148,7 @@ Input [4]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, ca_address_sk#9] Output [2]: [i_item_sk#12, i_item_id#13] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -156,13 +156,13 @@ Input [2]: [i_item_sk#12, i_item_id#13] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#12, i_item_id#13] -Condition : isnotnull(i_item_sk#12) +Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) (21) Scan parquet default.item Output [2]: [i_item_id#13, i_category#14] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music), IsNotNull(i_item_id)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -170,7 +170,7 @@ Input [2]: [i_item_id#13, i_category#14] (23) Filter [codegen id : 3] Input [2]: [i_item_id#13, i_category#14] -Condition : (isnotnull(i_category#14) AND (i_category#14 = Music)) +Condition : ((isnotnull(i_category#14) AND (i_category#14 = Music)) AND isnotnull(i_item_id#13)) (24) Project [codegen id : 3] Output [1]: [i_item_id#13 AS i_item_id#13#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index d49bf8669bea5..3eb81f44edd45 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] + Filter [i_item_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_id,i_item_sk] @@ -51,7 +51,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_category] + Filter [i_category,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_category,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt index e54aecbc37c22..e7b357cd4a3b8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt @@ -59,7 +59,7 @@ TakeOrderedAndProject (54) Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -67,7 +67,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -81,7 +81,7 @@ Arguments: [c_customer_sk#1 ASC NULLS FIRST], false, 0 Output [2]: [ss_sold_date_sk#5, ss_customer_sk#6] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] @@ -89,7 +89,7 @@ Input [2]: [ss_sold_date_sk#5, ss_customer_sk#6] (8) Filter [codegen id : 4] Input [2]: [ss_sold_date_sk#5, ss_customer_sk#6] -Condition : isnotnull(ss_sold_date_sk#5) +Condition : (isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_customer_sk#6)) (9) Scan parquet default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_moy#9] @@ -139,7 +139,7 @@ Join condition: None Output [2]: [ws_sold_date_sk#12, ws_bill_customer_sk#13] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (20) ColumnarToRow [codegen id : 7] @@ -147,7 +147,7 @@ Input [2]: [ws_sold_date_sk#12, ws_bill_customer_sk#13] (21) Filter [codegen id : 7] Input [2]: [ws_sold_date_sk#12, ws_bill_customer_sk#13] -Condition : isnotnull(ws_sold_date_sk#12) +Condition : (isnotnull(ws_sold_date_sk#12) AND isnotnull(ws_bill_customer_sk#13)) (22) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#7] @@ -178,7 +178,7 @@ Join condition: None Output [2]: [cs_sold_date_sk#15, cs_ship_customer_sk#16] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_customer_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 10] @@ -186,7 +186,7 @@ Input [2]: [cs_sold_date_sk#15, cs_ship_customer_sk#16] (30) Filter [codegen id : 10] Input [2]: [cs_sold_date_sk#15, cs_ship_customer_sk#16] -Condition : isnotnull(cs_sold_date_sk#15) +Condition : (isnotnull(cs_sold_date_sk#15) AND isnotnull(cs_ship_customer_sk#16)) (31) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt index 0d637a4674e69..687b50277f095 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] @@ -33,7 +33,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] @@ -52,7 +52,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital WholeStageCodegen (7) Project [ws_bill_customer_sk] BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_sold_date_sk] + Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] @@ -65,7 +65,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital WholeStageCodegen (10) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk] + Filter [cs_ship_customer_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index a5448b18bd4d8..4c27a2efe5da2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -54,7 +54,7 @@ TakeOrderedAndProject (49) Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -62,13 +62,13 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#4, ss_customer_sk#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] @@ -76,7 +76,7 @@ Input [2]: [ss_sold_date_sk#4, ss_customer_sk#5] (6) Filter [codegen id : 2] Input [2]: [ss_sold_date_sk#4, ss_customer_sk#5] -Condition : isnotnull(ss_sold_date_sk#4) +Condition : (isnotnull(ss_sold_date_sk#4) AND isnotnull(ss_customer_sk#5)) (7) Scan parquet default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] @@ -122,7 +122,7 @@ Join condition: None Output [2]: [ws_sold_date_sk#11, ws_bill_customer_sk#12] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 4] @@ -130,7 +130,7 @@ Input [2]: [ws_sold_date_sk#11, ws_bill_customer_sk#12] (18) Filter [codegen id : 4] Input [2]: [ws_sold_date_sk#11, ws_bill_customer_sk#12] -Condition : isnotnull(ws_sold_date_sk#11) +Condition : (isnotnull(ws_sold_date_sk#11) AND isnotnull(ws_bill_customer_sk#12)) (19) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#6] @@ -157,7 +157,7 @@ Join condition: None Output [2]: [cs_sold_date_sk#14, cs_ship_customer_sk#15] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_customer_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 6] @@ -165,7 +165,7 @@ Input [2]: [cs_sold_date_sk#14, cs_ship_customer_sk#15] (26) Filter [codegen id : 6] Input [2]: [cs_sold_date_sk#14, cs_ship_customer_sk#15] -Condition : isnotnull(cs_sold_date_sk#14) +Condition : (isnotnull(cs_sold_date_sk#14) AND isnotnull(cs_ship_customer_sk#15)) (27) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index 765e6a7524dea..de2e26d62d40c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] @@ -22,7 +22,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] @@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_sold_date_sk] + Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] @@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk] + Filter [cs_ship_customer_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index abca0d859fd17..eb8d9fac7efec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -97,7 +97,7 @@ Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3, d_date_sk#4] Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/store] -PushedFilters: [IsNotNull(s_store_sk)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -105,7 +105,7 @@ Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) +Condition : (isnotnull(s_store_sk#7) AND isnotnull(s_state#9)) (14) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3] @@ -192,7 +192,7 @@ Arguments: [rank(_w2#16) windowspecdefinition(s_state#9, _w2#16 DESC NULLS LAST, (32) Filter [codegen id : 7] Input [4]: [s_state#15, s_state#9, _w2#16, ranking#18] -Condition : (isnotnull(ranking#18) AND (ranking#18 <= 5)) +Condition : ((isnotnull(ranking#18) AND (ranking#18 <= 5)) AND isnotnull(s_state#15)) (33) Project [codegen id : 7] Output [1]: [s_state#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt index 1b14fd24d1aee..7e6819143c9c8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt @@ -34,7 +34,7 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su BroadcastExchange #4 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk] + Filter [s_state,s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_county,s_state,s_store_sk] @@ -42,7 +42,7 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su BroadcastExchange #5 WholeStageCodegen (7) Project [s_state] - Filter [ranking] + Filter [ranking,s_state] InputAdapter Window [_w2,s_state] WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 2e6b9ebdd0226..38fd1b7909a45 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -97,7 +97,7 @@ Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3, d_date_sk#4] Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/store] -PushedFilters: [IsNotNull(s_store_sk)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -105,7 +105,7 @@ Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) +Condition : (isnotnull(s_store_sk#7) AND isnotnull(s_state#9)) (14) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3] @@ -192,7 +192,7 @@ Arguments: [rank(_w2#16) windowspecdefinition(s_state#9, _w2#16 DESC NULLS LAST, (32) Filter [codegen id : 7] Input [4]: [s_state#15, s_state#9, _w2#16, ranking#18] -Condition : (isnotnull(ranking#18) AND (ranking#18 <= 5)) +Condition : ((isnotnull(ranking#18) AND (ranking#18 <= 5)) AND isnotnull(s_state#15)) (33) Project [codegen id : 7] Output [1]: [s_state#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index 1587213842374..1098d50936195 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -34,7 +34,7 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su BroadcastExchange #4 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk] + Filter [s_state,s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_county,s_state,s_store_sk] @@ -42,7 +42,7 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su BroadcastExchange #5 WholeStageCodegen (7) Project [s_state] - Filter [ranking] + Filter [ranking,s_state] InputAdapter Window [_w2,s_state] WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index 86a1fade375f2..f236c3a389b31 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -1,65 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (61) -+- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * HashAggregate (30) - : : +- Exchange (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) +TakeOrderedAndProject (62) ++- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_returns (1) - : : : +- BroadcastExchange (19) - : : : +- * Project (18) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (17) + : : : +- BroadcastExchange (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (18) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (14) - : : : :- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (13) - : : : +- * Project (12) - : : : +- * Filter (11) - : : : +- * ColumnarToRow (10) - : : : +- Scan parquet default.date_dim (9) - : : +- BroadcastExchange (25) - : : +- * Filter (24) - : : +- * ColumnarToRow (23) - : : +- Scan parquet default.item (22) - : +- BroadcastExchange (43) - : +- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Filter (33) - : : : +- * ColumnarToRow (32) - : : : +- Scan parquet default.catalog_returns (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- BroadcastExchange (58) - +- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet default.web_returns (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) + : : : +- BroadcastExchange (17) + : : : +- * Project (16) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (15) + : : : :- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.date_dim (7) + : : : +- BroadcastExchange (14) + : : : +- * Project (13) + : : : +- * Filter (12) + : : : +- * ColumnarToRow (11) + : : : +- Scan parquet default.date_dim (10) + : : +- BroadcastExchange (26) + : : +- * Filter (25) + : : +- * ColumnarToRow (24) + : : +- Scan parquet default.item (23) + : +- BroadcastExchange (44) + : +- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Filter (34) + : : : +- * ColumnarToRow (33) + : : : +- Scan parquet default.catalog_returns (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- BroadcastExchange (59) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Filter (49) + : : +- * ColumnarToRow (48) + : : +- Scan parquet default.web_returns (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) (1) Scan parquet default.store_returns @@ -80,7 +81,7 @@ Condition : (isnotnull(sr_item_sk#2) AND isnotnull(sr_returned_date_sk#1)) Output [2]: [d_date_sk#4, d_date#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (5) ColumnarToRow [codegen id : 3] @@ -88,257 +89,263 @@ Input [2]: [d_date_sk#4, d_date#5] (6) Filter [codegen id : 3] Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_date#5)) (7) Scan parquet default.date_dim Output [2]: [d_date#5, d_week_seq#6] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (8) ColumnarToRow [codegen id : 2] Input [2]: [d_date#5, d_week_seq#6] -(9) Scan parquet default.date_dim +(9) Filter [codegen id : 2] +Input [2]: [d_date#5, d_week_seq#6] +Condition : (isnotnull(d_week_seq#6) AND isnotnull(d_date#5)) + +(10) Scan parquet default.date_dim Output [2]: [d_date#5, d_week_seq#6] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/date_dim] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 1] +(11) ColumnarToRow [codegen id : 1] Input [2]: [d_date#5, d_week_seq#6] -(11) Filter [codegen id : 1] +(12) Filter [codegen id : 1] Input [2]: [d_date#5, d_week_seq#6] -Condition : cast(d_date#5 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Condition : (cast(d_date#5 as string) IN (2000-06-30,2000-09-27,2000-11-17) AND isnotnull(d_week_seq#6)) -(12) Project [codegen id : 1] +(13) Project [codegen id : 1] Output [1]: [d_week_seq#6 AS d_week_seq#6#7] Input [2]: [d_date#5, d_week_seq#6] -(13) BroadcastExchange +(14) BroadcastExchange Input [1]: [d_week_seq#6#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] -(14) BroadcastHashJoin [codegen id : 2] +(15) BroadcastHashJoin [codegen id : 2] Left keys [1]: [d_week_seq#6] Right keys [1]: [d_week_seq#6#7] Join condition: None -(15) Project [codegen id : 2] +(16) Project [codegen id : 2] Output [1]: [d_date#5 AS d_date#5#9] Input [2]: [d_date#5, d_week_seq#6] -(16) BroadcastExchange +(17) BroadcastExchange Input [1]: [d_date#5#9] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] -(17) BroadcastHashJoin [codegen id : 3] +(18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_date#5] Right keys [1]: [d_date#5#9] Join condition: None -(18) Project [codegen id : 3] +(19) Project [codegen id : 3] Output [1]: [d_date_sk#4] Input [2]: [d_date_sk#4, d_date#5] -(19) BroadcastExchange +(20) BroadcastExchange Input [1]: [d_date_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(20) BroadcastHashJoin [codegen id : 5] +(21) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#1] Right keys [1]: [cast(d_date_sk#4 as bigint)] Join condition: None -(21) Project [codegen id : 5] +(22) Project [codegen id : 5] Output [2]: [sr_item_sk#2, sr_return_quantity#3] Input [4]: [sr_returned_date_sk#1, sr_item_sk#2, sr_return_quantity#3, d_date_sk#4] -(22) Scan parquet default.item +(23) Scan parquet default.item Output [2]: [i_item_sk#12, i_item_id#13] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 4] Input [2]: [i_item_sk#12, i_item_id#13] -(24) Filter [codegen id : 4] +(25) Filter [codegen id : 4] Input [2]: [i_item_sk#12, i_item_id#13] Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) -(25) BroadcastExchange +(26) BroadcastExchange Input [2]: [i_item_sk#12, i_item_id#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] -(26) BroadcastHashJoin [codegen id : 5] +(27) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#2] Right keys [1]: [cast(i_item_sk#12 as bigint)] Join condition: None -(27) Project [codegen id : 5] +(28) Project [codegen id : 5] Output [2]: [sr_return_quantity#3, i_item_id#13] Input [4]: [sr_item_sk#2, sr_return_quantity#3, i_item_sk#12, i_item_id#13] -(28) HashAggregate [codegen id : 5] +(29) HashAggregate [codegen id : 5] Input [2]: [sr_return_quantity#3, i_item_id#13] Keys [1]: [i_item_id#13] Functions [1]: [partial_sum(cast(sr_return_quantity#3 as bigint))] Aggregate Attributes [1]: [sum#15] Results [2]: [i_item_id#13, sum#16] -(29) Exchange +(30) Exchange Input [2]: [i_item_id#13, sum#16] Arguments: hashpartitioning(i_item_id#13, 5), true, [id=#17] -(30) HashAggregate [codegen id : 18] +(31) HashAggregate [codegen id : 18] Input [2]: [i_item_id#13, sum#16] Keys [1]: [i_item_id#13] Functions [1]: [sum(cast(sr_return_quantity#3 as bigint))] Aggregate Attributes [1]: [sum(cast(sr_return_quantity#3 as bigint))#18] Results [2]: [i_item_id#13 AS item_id#19, sum(cast(sr_return_quantity#3 as bigint))#18 AS sr_item_qty#20] -(31) Scan parquet default.catalog_returns +(32) Scan parquet default.catalog_returns Output [3]: [cr_returned_date_sk#21, cr_item_sk#22, cr_return_quantity#23] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_returned_date_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] +(33) ColumnarToRow [codegen id : 10] Input [3]: [cr_returned_date_sk#21, cr_item_sk#22, cr_return_quantity#23] -(33) Filter [codegen id : 10] +(34) Filter [codegen id : 10] Input [3]: [cr_returned_date_sk#21, cr_item_sk#22, cr_return_quantity#23] Condition : (isnotnull(cr_item_sk#22) AND isnotnull(cr_returned_date_sk#21)) -(34) ReusedExchange [Reuses operator id: 19] +(35) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#4] -(35) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_returned_date_sk#21] Right keys [1]: [d_date_sk#4] Join condition: None -(36) Project [codegen id : 10] +(37) Project [codegen id : 10] Output [2]: [cr_item_sk#22, cr_return_quantity#23] Input [4]: [cr_returned_date_sk#21, cr_item_sk#22, cr_return_quantity#23, d_date_sk#4] -(37) ReusedExchange [Reuses operator id: 25] +(38) ReusedExchange [Reuses operator id: 26] Output [2]: [i_item_sk#12, i_item_id#13] -(38) BroadcastHashJoin [codegen id : 10] +(39) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_item_sk#22] Right keys [1]: [i_item_sk#12] Join condition: None -(39) Project [codegen id : 10] +(40) Project [codegen id : 10] Output [2]: [cr_return_quantity#23, i_item_id#13] Input [4]: [cr_item_sk#22, cr_return_quantity#23, i_item_sk#12, i_item_id#13] -(40) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 10] Input [2]: [cr_return_quantity#23, i_item_id#13] Keys [1]: [i_item_id#13] Functions [1]: [partial_sum(cast(cr_return_quantity#23 as bigint))] Aggregate Attributes [1]: [sum#24] Results [2]: [i_item_id#13, sum#25] -(41) Exchange +(42) Exchange Input [2]: [i_item_id#13, sum#25] Arguments: hashpartitioning(i_item_id#13, 5), true, [id=#26] -(42) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 11] Input [2]: [i_item_id#13, sum#25] Keys [1]: [i_item_id#13] Functions [1]: [sum(cast(cr_return_quantity#23 as bigint))] Aggregate Attributes [1]: [sum(cast(cr_return_quantity#23 as bigint))#27] Results [2]: [i_item_id#13 AS item_id#28, sum(cast(cr_return_quantity#23 as bigint))#27 AS cr_item_qty#29] -(43) BroadcastExchange +(44) BroadcastExchange Input [2]: [item_id#28, cr_item_qty#29] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] -(44) BroadcastHashJoin [codegen id : 18] +(45) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#19] Right keys [1]: [item_id#28] Join condition: None -(45) Project [codegen id : 18] +(46) Project [codegen id : 18] Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] -(46) Scan parquet default.web_returns +(47) Scan parquet default.web_returns Output [3]: [wr_returned_date_sk#31, wr_item_sk#32, wr_return_quantity#33] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_returned_date_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 16] +(48) ColumnarToRow [codegen id : 16] Input [3]: [wr_returned_date_sk#31, wr_item_sk#32, wr_return_quantity#33] -(48) Filter [codegen id : 16] +(49) Filter [codegen id : 16] Input [3]: [wr_returned_date_sk#31, wr_item_sk#32, wr_return_quantity#33] Condition : (isnotnull(wr_item_sk#32) AND isnotnull(wr_returned_date_sk#31)) -(49) ReusedExchange [Reuses operator id: 19] +(50) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#4] -(50) BroadcastHashJoin [codegen id : 16] +(51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [wr_returned_date_sk#31] Right keys [1]: [cast(d_date_sk#4 as bigint)] Join condition: None -(51) Project [codegen id : 16] +(52) Project [codegen id : 16] Output [2]: [wr_item_sk#32, wr_return_quantity#33] Input [4]: [wr_returned_date_sk#31, wr_item_sk#32, wr_return_quantity#33, d_date_sk#4] -(52) ReusedExchange [Reuses operator id: 25] +(53) ReusedExchange [Reuses operator id: 26] Output [2]: [i_item_sk#12, i_item_id#13] -(53) BroadcastHashJoin [codegen id : 16] +(54) BroadcastHashJoin [codegen id : 16] Left keys [1]: [wr_item_sk#32] Right keys [1]: [cast(i_item_sk#12 as bigint)] Join condition: None -(54) Project [codegen id : 16] +(55) Project [codegen id : 16] Output [2]: [wr_return_quantity#33, i_item_id#13] Input [4]: [wr_item_sk#32, wr_return_quantity#33, i_item_sk#12, i_item_id#13] -(55) HashAggregate [codegen id : 16] +(56) HashAggregate [codegen id : 16] Input [2]: [wr_return_quantity#33, i_item_id#13] Keys [1]: [i_item_id#13] Functions [1]: [partial_sum(cast(wr_return_quantity#33 as bigint))] Aggregate Attributes [1]: [sum#34] Results [2]: [i_item_id#13, sum#35] -(56) Exchange +(57) Exchange Input [2]: [i_item_id#13, sum#35] Arguments: hashpartitioning(i_item_id#13, 5), true, [id=#36] -(57) HashAggregate [codegen id : 17] +(58) HashAggregate [codegen id : 17] Input [2]: [i_item_id#13, sum#35] Keys [1]: [i_item_id#13] Functions [1]: [sum(cast(wr_return_quantity#33 as bigint))] Aggregate Attributes [1]: [sum(cast(wr_return_quantity#33 as bigint))#37] Results [2]: [i_item_id#13 AS item_id#38, sum(cast(wr_return_quantity#33 as bigint))#37 AS wr_item_qty#39] -(58) BroadcastExchange +(59) BroadcastExchange Input [2]: [item_id#38, wr_item_qty#39] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] -(59) BroadcastHashJoin [codegen id : 18] +(60) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#19] Right keys [1]: [item_id#38] Join condition: None -(60) Project [codegen id : 18] +(61) Project [codegen id : 18] Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] -(61) TakeOrderedAndProject +(62) TakeOrderedAndProject Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt index 294bc6bea32d6..6f694b8952329 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject [average,cr_dev,cr_item_qty,item_id,sr_dev,sr_item_qty,wr_ WholeStageCodegen (3) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_date_sk] @@ -31,14 +31,15 @@ TakeOrderedAndProject [average,cr_dev,cr_item_qty,item_id,sr_dev,sr_item_qty,wr_ WholeStageCodegen (2) Project [d_date] BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + Filter [d_date,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_week_seq] - Filter [d_date] + Filter [d_date,d_week_seq] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index 39fc20878d286..717c10d8a8b21 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,14 +1,14 @@ == Physical Plan == -TakeOrderedAndProject (61) -+- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * HashAggregate (30) - : : +- Exchange (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) +TakeOrderedAndProject (62) ++- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) : : :- * Project (9) : : : +- * BroadcastHashJoin Inner BuildRight (8) : : : :- * Filter (3) @@ -18,48 +18,49 @@ TakeOrderedAndProject (61) : : : +- * Filter (6) : : : +- * ColumnarToRow (5) : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin LeftSemi BuildRight (24) : : :- * Filter (12) : : : +- * ColumnarToRow (11) : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- * BroadcastHashJoin LeftSemi BuildRight (20) - : : :- * ColumnarToRow (14) - : : : +- Scan parquet default.date_dim (13) - : : +- BroadcastExchange (19) - : : +- * Project (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.date_dim (15) - : +- BroadcastExchange (43) - : +- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Filter (33) - : : : +- * ColumnarToRow (32) - : : : +- Scan parquet default.catalog_returns (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- BroadcastExchange (58) - +- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet default.web_returns (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin LeftSemi BuildRight (21) + : : :- * Filter (15) + : : : +- * ColumnarToRow (14) + : : : +- Scan parquet default.date_dim (13) + : : +- BroadcastExchange (20) + : : +- * Project (19) + : : +- * Filter (18) + : : +- * ColumnarToRow (17) + : : +- Scan parquet default.date_dim (16) + : +- BroadcastExchange (44) + : +- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Filter (34) + : : : +- * ColumnarToRow (33) + : : : +- Scan parquet default.catalog_returns (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- BroadcastExchange (59) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Filter (49) + : : +- * ColumnarToRow (48) + : : +- Scan parquet default.web_returns (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) (1) Scan parquet default.store_returns @@ -107,7 +108,7 @@ Input [5]: [sr_returned_date_sk#1, sr_item_sk#2, sr_return_quantity#3, i_item_sk Output [2]: [d_date_sk#7, d_date#8] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (11) ColumnarToRow [codegen id : 4] @@ -115,230 +116,236 @@ Input [2]: [d_date_sk#7, d_date#8] (12) Filter [codegen id : 4] Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Condition : (isnotnull(d_date_sk#7) AND isnotnull(d_date#8)) (13) Scan parquet default.date_dim Output [2]: [d_date#8, d_week_seq#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] Input [2]: [d_date#8, d_week_seq#9] -(15) Scan parquet default.date_dim +(15) Filter [codegen id : 3] +Input [2]: [d_date#8, d_week_seq#9] +Condition : (isnotnull(d_week_seq#9) AND isnotnull(d_date#8)) + +(16) Scan parquet default.date_dim Output [2]: [d_date#8, d_week_seq#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/date_dim] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 2] +(17) ColumnarToRow [codegen id : 2] Input [2]: [d_date#8, d_week_seq#9] -(17) Filter [codegen id : 2] +(18) Filter [codegen id : 2] Input [2]: [d_date#8, d_week_seq#9] -Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Condition : (cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) AND isnotnull(d_week_seq#9)) -(18) Project [codegen id : 2] +(19) Project [codegen id : 2] Output [1]: [d_week_seq#9 AS d_week_seq#9#10] Input [2]: [d_date#8, d_week_seq#9] -(19) BroadcastExchange +(20) BroadcastExchange Input [1]: [d_week_seq#9#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(20) BroadcastHashJoin [codegen id : 3] +(21) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_week_seq#9] Right keys [1]: [d_week_seq#9#10] Join condition: None -(21) Project [codegen id : 3] +(22) Project [codegen id : 3] Output [1]: [d_date#8 AS d_date#8#12] Input [2]: [d_date#8, d_week_seq#9] -(22) BroadcastExchange +(23) BroadcastExchange Input [1]: [d_date#8#12] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] -(23) BroadcastHashJoin [codegen id : 4] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [d_date#8] Right keys [1]: [d_date#8#12] Join condition: None -(24) Project [codegen id : 4] +(25) Project [codegen id : 4] Output [1]: [d_date_sk#7] Input [2]: [d_date_sk#7, d_date#8] -(25) BroadcastExchange +(26) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(26) BroadcastHashJoin [codegen id : 5] +(27) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#1] Right keys [1]: [cast(d_date_sk#7 as bigint)] Join condition: None -(27) Project [codegen id : 5] +(28) Project [codegen id : 5] Output [2]: [sr_return_quantity#3, i_item_id#5] Input [4]: [sr_returned_date_sk#1, sr_return_quantity#3, i_item_id#5, d_date_sk#7] -(28) HashAggregate [codegen id : 5] +(29) HashAggregate [codegen id : 5] Input [2]: [sr_return_quantity#3, i_item_id#5] Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(cast(sr_return_quantity#3 as bigint))] Aggregate Attributes [1]: [sum#15] Results [2]: [i_item_id#5, sum#16] -(29) Exchange +(30) Exchange Input [2]: [i_item_id#5, sum#16] Arguments: hashpartitioning(i_item_id#5, 5), true, [id=#17] -(30) HashAggregate [codegen id : 18] +(31) HashAggregate [codegen id : 18] Input [2]: [i_item_id#5, sum#16] Keys [1]: [i_item_id#5] Functions [1]: [sum(cast(sr_return_quantity#3 as bigint))] Aggregate Attributes [1]: [sum(cast(sr_return_quantity#3 as bigint))#18] Results [2]: [i_item_id#5 AS item_id#19, sum(cast(sr_return_quantity#3 as bigint))#18 AS sr_item_qty#20] -(31) Scan parquet default.catalog_returns +(32) Scan parquet default.catalog_returns Output [3]: [cr_returned_date_sk#21, cr_item_sk#22, cr_return_quantity#23] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_returned_date_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] +(33) ColumnarToRow [codegen id : 10] Input [3]: [cr_returned_date_sk#21, cr_item_sk#22, cr_return_quantity#23] -(33) Filter [codegen id : 10] +(34) Filter [codegen id : 10] Input [3]: [cr_returned_date_sk#21, cr_item_sk#22, cr_return_quantity#23] Condition : (isnotnull(cr_item_sk#22) AND isnotnull(cr_returned_date_sk#21)) -(34) ReusedExchange [Reuses operator id: 7] +(35) ReusedExchange [Reuses operator id: 7] Output [2]: [i_item_sk#4, i_item_id#5] -(35) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_item_sk#22] Right keys [1]: [i_item_sk#4] Join condition: None -(36) Project [codegen id : 10] +(37) Project [codegen id : 10] Output [3]: [cr_returned_date_sk#21, cr_return_quantity#23, i_item_id#5] Input [5]: [cr_returned_date_sk#21, cr_item_sk#22, cr_return_quantity#23, i_item_sk#4, i_item_id#5] -(37) ReusedExchange [Reuses operator id: 25] +(38) ReusedExchange [Reuses operator id: 26] Output [1]: [d_date_sk#7] -(38) BroadcastHashJoin [codegen id : 10] +(39) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_returned_date_sk#21] Right keys [1]: [d_date_sk#7] Join condition: None -(39) Project [codegen id : 10] +(40) Project [codegen id : 10] Output [2]: [cr_return_quantity#23, i_item_id#5] Input [4]: [cr_returned_date_sk#21, cr_return_quantity#23, i_item_id#5, d_date_sk#7] -(40) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 10] Input [2]: [cr_return_quantity#23, i_item_id#5] Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(cast(cr_return_quantity#23 as bigint))] Aggregate Attributes [1]: [sum#24] Results [2]: [i_item_id#5, sum#25] -(41) Exchange +(42) Exchange Input [2]: [i_item_id#5, sum#25] Arguments: hashpartitioning(i_item_id#5, 5), true, [id=#26] -(42) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 11] Input [2]: [i_item_id#5, sum#25] Keys [1]: [i_item_id#5] Functions [1]: [sum(cast(cr_return_quantity#23 as bigint))] Aggregate Attributes [1]: [sum(cast(cr_return_quantity#23 as bigint))#27] Results [2]: [i_item_id#5 AS item_id#28, sum(cast(cr_return_quantity#23 as bigint))#27 AS cr_item_qty#29] -(43) BroadcastExchange +(44) BroadcastExchange Input [2]: [item_id#28, cr_item_qty#29] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] -(44) BroadcastHashJoin [codegen id : 18] +(45) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#19] Right keys [1]: [item_id#28] Join condition: None -(45) Project [codegen id : 18] +(46) Project [codegen id : 18] Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] -(46) Scan parquet default.web_returns +(47) Scan parquet default.web_returns Output [3]: [wr_returned_date_sk#31, wr_item_sk#32, wr_return_quantity#33] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_returned_date_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 16] +(48) ColumnarToRow [codegen id : 16] Input [3]: [wr_returned_date_sk#31, wr_item_sk#32, wr_return_quantity#33] -(48) Filter [codegen id : 16] +(49) Filter [codegen id : 16] Input [3]: [wr_returned_date_sk#31, wr_item_sk#32, wr_return_quantity#33] Condition : (isnotnull(wr_item_sk#32) AND isnotnull(wr_returned_date_sk#31)) -(49) ReusedExchange [Reuses operator id: 7] +(50) ReusedExchange [Reuses operator id: 7] Output [2]: [i_item_sk#4, i_item_id#5] -(50) BroadcastHashJoin [codegen id : 16] +(51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [wr_item_sk#32] Right keys [1]: [cast(i_item_sk#4 as bigint)] Join condition: None -(51) Project [codegen id : 16] +(52) Project [codegen id : 16] Output [3]: [wr_returned_date_sk#31, wr_return_quantity#33, i_item_id#5] Input [5]: [wr_returned_date_sk#31, wr_item_sk#32, wr_return_quantity#33, i_item_sk#4, i_item_id#5] -(52) ReusedExchange [Reuses operator id: 25] +(53) ReusedExchange [Reuses operator id: 26] Output [1]: [d_date_sk#7] -(53) BroadcastHashJoin [codegen id : 16] +(54) BroadcastHashJoin [codegen id : 16] Left keys [1]: [wr_returned_date_sk#31] Right keys [1]: [cast(d_date_sk#7 as bigint)] Join condition: None -(54) Project [codegen id : 16] +(55) Project [codegen id : 16] Output [2]: [wr_return_quantity#33, i_item_id#5] Input [4]: [wr_returned_date_sk#31, wr_return_quantity#33, i_item_id#5, d_date_sk#7] -(55) HashAggregate [codegen id : 16] +(56) HashAggregate [codegen id : 16] Input [2]: [wr_return_quantity#33, i_item_id#5] Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(cast(wr_return_quantity#33 as bigint))] Aggregate Attributes [1]: [sum#34] Results [2]: [i_item_id#5, sum#35] -(56) Exchange +(57) Exchange Input [2]: [i_item_id#5, sum#35] Arguments: hashpartitioning(i_item_id#5, 5), true, [id=#36] -(57) HashAggregate [codegen id : 17] +(58) HashAggregate [codegen id : 17] Input [2]: [i_item_id#5, sum#35] Keys [1]: [i_item_id#5] Functions [1]: [sum(cast(wr_return_quantity#33 as bigint))] Aggregate Attributes [1]: [sum(cast(wr_return_quantity#33 as bigint))#37] Results [2]: [i_item_id#5 AS item_id#38, sum(cast(wr_return_quantity#33 as bigint))#37 AS wr_item_qty#39] -(58) BroadcastExchange +(59) BroadcastExchange Input [2]: [item_id#38, wr_item_qty#39] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] -(59) BroadcastHashJoin [codegen id : 18] +(60) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#19] Right keys [1]: [item_id#38] Join condition: None -(60) Project [codegen id : 18] +(61) Project [codegen id : 18] Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] -(61) TakeOrderedAndProject +(62) TakeOrderedAndProject Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index d0639eada79ae..97e1b70fcaec0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -29,7 +29,7 @@ TakeOrderedAndProject [average,cr_dev,cr_item_qty,item_id,sr_dev,sr_item_qty,wr_ WholeStageCodegen (4) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_date_sk] @@ -38,14 +38,15 @@ TakeOrderedAndProject [average,cr_dev,cr_item_qty,item_id,sr_dev,sr_item_qty,wr_ WholeStageCodegen (3) Project [d_date] BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + Filter [d_date,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) Project [d_week_seq] - Filter [d_date] + Filter [d_date,d_week_seq] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt index b71f2ab6ae3c1..f64af65811df8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (22) -+- * HashAggregate (21) - +- Exchange (20) - +- * HashAggregate (19) - +- * Project (18) - +- * SortMergeJoin Inner (17) +TakeOrderedAndProject (23) ++- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Project (19) + +- * SortMergeJoin Inner (18) :- * Sort (12) : +- Exchange (11) : +- * Project (10) @@ -17,10 +17,11 @@ TakeOrderedAndProject (22) : +- * Filter (6) : +- * ColumnarToRow (5) : +- Scan parquet default.reason (4) - +- * Sort (16) - +- Exchange (15) - +- * ColumnarToRow (14) - +- Scan parquet default.store_sales (13) + +- * Sort (17) + +- Exchange (16) + +- * Filter (15) + +- * ColumnarToRow (14) + +- Scan parquet default.store_sales (13) (1) Scan parquet default.store_returns @@ -80,47 +81,52 @@ Arguments: [sr_item_sk#1 ASC NULLS FIRST, sr_ticket_number#3 ASC NULLS FIRST], f Output [5]: [ss_item_sk#9, ss_customer_sk#10, ss_ticket_number#11, ss_quantity#12, ss_sales_price#13] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] Input [5]: [ss_item_sk#9, ss_customer_sk#10, ss_ticket_number#11, ss_quantity#12, ss_sales_price#13] -(15) Exchange +(15) Filter [codegen id : 4] +Input [5]: [ss_item_sk#9, ss_customer_sk#10, ss_ticket_number#11, ss_quantity#12, ss_sales_price#13] +Condition : (isnotnull(ss_item_sk#9) AND isnotnull(ss_ticket_number#11)) + +(16) Exchange Input [5]: [ss_item_sk#9, ss_customer_sk#10, ss_ticket_number#11, ss_quantity#12, ss_sales_price#13] Arguments: hashpartitioning(cast(ss_item_sk#9 as bigint), cast(ss_ticket_number#11 as bigint), 5), true, [id=#14] -(16) Sort [codegen id : 5] +(17) Sort [codegen id : 5] Input [5]: [ss_item_sk#9, ss_customer_sk#10, ss_ticket_number#11, ss_quantity#12, ss_sales_price#13] Arguments: [cast(ss_item_sk#9 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#11 as bigint) ASC NULLS FIRST], false, 0 -(17) SortMergeJoin [codegen id : 6] +(18) SortMergeJoin [codegen id : 6] Left keys [2]: [sr_item_sk#1, sr_ticket_number#3] Right keys [2]: [cast(ss_item_sk#9 as bigint), cast(ss_ticket_number#11 as bigint)] Join condition: None -(18) Project [codegen id : 6] +(19) Project [codegen id : 6] Output [2]: [ss_customer_sk#10, CASE WHEN isnotnull(sr_return_quantity#4) THEN CheckOverflow((promote_precision(cast(cast((ss_quantity#12 - sr_return_quantity#4) as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#13 as decimal(12,2)))), DecimalType(18,2), true) ELSE CheckOverflow((promote_precision(cast(cast(ss_quantity#12 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#13 as decimal(12,2)))), DecimalType(18,2), true) END AS act_sales#15] Input [8]: [sr_item_sk#1, sr_ticket_number#3, sr_return_quantity#4, ss_item_sk#9, ss_customer_sk#10, ss_ticket_number#11, ss_quantity#12, ss_sales_price#13] -(19) HashAggregate [codegen id : 6] +(20) HashAggregate [codegen id : 6] Input [2]: [ss_customer_sk#10, act_sales#15] Keys [1]: [ss_customer_sk#10] Functions [1]: [partial_sum(act_sales#15)] Aggregate Attributes [2]: [sum#16, isEmpty#17] Results [3]: [ss_customer_sk#10, sum#18, isEmpty#19] -(20) Exchange +(21) Exchange Input [3]: [ss_customer_sk#10, sum#18, isEmpty#19] Arguments: hashpartitioning(ss_customer_sk#10, 5), true, [id=#20] -(21) HashAggregate [codegen id : 7] +(22) HashAggregate [codegen id : 7] Input [3]: [ss_customer_sk#10, sum#18, isEmpty#19] Keys [1]: [ss_customer_sk#10] Functions [1]: [sum(act_sales#15)] Aggregate Attributes [1]: [sum(act_sales#15)#21] Results [2]: [ss_customer_sk#10, sum(act_sales#15)#21 AS sumsales#22] -(22) TakeOrderedAndProject +(23) TakeOrderedAndProject Input [2]: [ss_customer_sk#10, sumsales#22] Arguments: 100, [sumsales#22 ASC NULLS FIRST, ss_customer_sk#10 ASC NULLS FIRST], [ss_customer_sk#10, sumsales#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt index e1144076af8c6..7ec2694d371ee 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt @@ -33,6 +33,7 @@ TakeOrderedAndProject [ss_customer_sk,sumsales] InputAdapter Exchange [ss_item_sk,ss_ticket_number] #4 WholeStageCodegen (4) - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_ticket_number] + Filter [ss_item_sk,ss_ticket_number] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_ticket_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index 496c2ab591de4..df1b05947bc5a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -1,111 +1,117 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- Exchange (17) - +- * HashAggregate (16) - +- * Project (15) - +- * BroadcastHashJoin Inner BuildRight (14) - :- * Project (8) - : +- * BroadcastHashJoin Inner BuildRight (7) - : :- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- BroadcastExchange (6) - : +- * Filter (5) - : +- * ColumnarToRow (4) - : +- Scan parquet default.store_returns (3) - +- BroadcastExchange (13) - +- * Project (12) - +- * Filter (11) - +- * ColumnarToRow (10) - +- Scan parquet default.reason (9) +TakeOrderedAndProject (20) ++- * HashAggregate (19) + +- Exchange (18) + +- * HashAggregate (17) + +- * Project (16) + +- * BroadcastHashJoin Inner BuildRight (15) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.store_sales (1) + : +- BroadcastExchange (7) + : +- * Filter (6) + : +- * ColumnarToRow (5) + : +- Scan parquet default.store_returns (4) + +- BroadcastExchange (14) + +- * Project (13) + +- * Filter (12) + +- * ColumnarToRow (11) + +- Scan parquet default.reason (10) (1) Scan parquet default.store_sales Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number)] ReadSchema: struct (2) ColumnarToRow [codegen id : 3] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -(3) Scan parquet default.store_returns +(3) Filter [codegen id : 3] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] +Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#3)) + +(4) Scan parquet default.store_returns Output [4]: [sr_item_sk#6, sr_reason_sk#7, sr_ticket_number#8, sr_return_quantity#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] ReadSchema: struct -(4) ColumnarToRow [codegen id : 1] +(5) ColumnarToRow [codegen id : 1] Input [4]: [sr_item_sk#6, sr_reason_sk#7, sr_ticket_number#8, sr_return_quantity#9] -(5) Filter [codegen id : 1] +(6) Filter [codegen id : 1] Input [4]: [sr_item_sk#6, sr_reason_sk#7, sr_ticket_number#8, sr_return_quantity#9] Condition : ((isnotnull(sr_item_sk#6) AND isnotnull(sr_ticket_number#8)) AND isnotnull(sr_reason_sk#7)) -(6) BroadcastExchange +(7) BroadcastExchange Input [4]: [sr_item_sk#6, sr_reason_sk#7, sr_ticket_number#8, sr_return_quantity#9] Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false], input[2, bigint, false]),false), [id=#10] -(7) BroadcastHashJoin [codegen id : 3] +(8) BroadcastHashJoin [codegen id : 3] Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#3 as bigint)] Right keys [2]: [sr_item_sk#6, sr_ticket_number#8] Join condition: None -(8) Project [codegen id : 3] +(9) Project [codegen id : 3] Output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#7, sr_return_quantity#9] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#6, sr_reason_sk#7, sr_ticket_number#8, sr_return_quantity#9] -(9) Scan parquet default.reason +(10) Scan parquet default.reason Output [2]: [r_reason_sk#11, r_reason_desc#12] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/reason] PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28), IsNotNull(r_reason_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 2] +(11) ColumnarToRow [codegen id : 2] Input [2]: [r_reason_sk#11, r_reason_desc#12] -(11) Filter [codegen id : 2] +(12) Filter [codegen id : 2] Input [2]: [r_reason_sk#11, r_reason_desc#12] Condition : ((isnotnull(r_reason_desc#12) AND (r_reason_desc#12 = reason 28)) AND isnotnull(r_reason_sk#11)) -(12) Project [codegen id : 2] +(13) Project [codegen id : 2] Output [1]: [r_reason_sk#11] Input [2]: [r_reason_sk#11, r_reason_desc#12] -(13) BroadcastExchange +(14) BroadcastExchange Input [1]: [r_reason_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(14) BroadcastHashJoin [codegen id : 3] +(15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [sr_reason_sk#7] Right keys [1]: [cast(r_reason_sk#11 as bigint)] Join condition: None -(15) Project [codegen id : 3] +(16) Project [codegen id : 3] Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#9) THEN CheckOverflow((promote_precision(cast(cast((ss_quantity#4 - sr_return_quantity#9) as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#5 as decimal(12,2)))), DecimalType(18,2), true) ELSE CheckOverflow((promote_precision(cast(cast(ss_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#5 as decimal(12,2)))), DecimalType(18,2), true) END AS act_sales#14] Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#7, sr_return_quantity#9, r_reason_sk#11] -(16) HashAggregate [codegen id : 3] +(17) HashAggregate [codegen id : 3] Input [2]: [ss_customer_sk#2, act_sales#14] Keys [1]: [ss_customer_sk#2] Functions [1]: [partial_sum(act_sales#14)] Aggregate Attributes [2]: [sum#15, isEmpty#16] Results [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -(17) Exchange +(18) Exchange Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#19] -(18) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 4] Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] Keys [1]: [ss_customer_sk#2] Functions [1]: [sum(act_sales#14)] Aggregate Attributes [1]: [sum(act_sales#14)#20] Results [2]: [ss_customer_sk#2, sum(act_sales#14)#20 AS sumsales#21] -(19) TakeOrderedAndProject +(20) TakeOrderedAndProject Input [2]: [ss_customer_sk#2, sumsales#21] Arguments: 100, [sumsales#21 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#21] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index 05e17cc4b3bd8..7511c2363db3a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -9,9 +9,10 @@ TakeOrderedAndProject [ss_customer_sk,sumsales] BroadcastHashJoin [r_reason_sk,sr_reason_sk] Project [sr_reason_sk,sr_return_quantity,ss_customer_sk,ss_quantity,ss_sales_price] BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_ticket_number] + Filter [ss_item_sk,ss_ticket_number] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_ticket_number] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index aed3635e09bfe..25a225198ebe7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -1,58 +1,60 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- SortMergeJoin LeftAnti (19) - : : : :- * Sort (14) - : : : : +- Exchange (13) - : : : : +- * Project (12) - : : : : +- SortMergeJoin LeftSemi (11) +TakeOrderedAndProject (49) ++- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- SortMergeJoin LeftAnti (21) + : : : :- * Sort (15) + : : : : +- Exchange (14) + : : : : +- * Project (13) + : : : : +- SortMergeJoin LeftSemi (12) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.web_sales (1) - : : : : +- * Sort (10) - : : : : +- Exchange (9) - : : : : +- * Project (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.web_sales (6) - : : : +- * Sort (18) - : : : +- Exchange (17) - : : : +- * ColumnarToRow (16) - : : : +- Scan parquet default.web_returns (15) - : : +- BroadcastExchange (24) - : : +- * Project (23) - : : +- * Filter (22) - : : +- * ColumnarToRow (21) - : : +- Scan parquet default.customer_address (20) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet default.web_site (27) - +- BroadcastExchange (38) - +- * Project (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.date_dim (34) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * Project (9) + : : : : +- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.web_sales (6) + : : : +- * Sort (20) + : : : +- Exchange (19) + : : : +- * Filter (18) + : : : +- * ColumnarToRow (17) + : : : +- Scan parquet default.web_returns (16) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet default.customer_address (22) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet default.web_site (29) + +- BroadcastExchange (40) + +- * Project (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet default.date_dim (36) (1) Scan parquet default.web_sales Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_warehouse_sk), IsNotNull(ws_order_number)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -60,7 +62,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (3) Filter [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) +Condition : ((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_warehouse_sk#4)) AND isnotnull(ws_order_number#5)) (4) Exchange Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -74,192 +76,202 @@ Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 Output [2]: [ws_warehouse_sk#4, ws_order_number#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/web_sales] +PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 3] Input [2]: [ws_warehouse_sk#4, ws_order_number#5] -(8) Project [codegen id : 3] +(8) Filter [codegen id : 3] +Input [2]: [ws_warehouse_sk#4, ws_order_number#5] +Condition : (isnotnull(ws_order_number#5) AND isnotnull(ws_warehouse_sk#4)) + +(9) Project [codegen id : 3] Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#9, ws_order_number#5 AS ws_order_number#5#10] Input [2]: [ws_warehouse_sk#4, ws_order_number#5] -(9) Exchange +(10) Exchange Input [2]: [ws_warehouse_sk#4#9, ws_order_number#5#10] Arguments: hashpartitioning(ws_order_number#5#10, 5), true, [id=#11] -(10) Sort [codegen id : 4] +(11) Sort [codegen id : 4] Input [2]: [ws_warehouse_sk#4#9, ws_order_number#5#10] Arguments: [ws_order_number#5#10 ASC NULLS FIRST], false, 0 -(11) SortMergeJoin +(12) SortMergeJoin Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#5#10] Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#9) -(12) Project [codegen id : 5] +(13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(13) Exchange +(14) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), true, [id=#12] -(14) Sort [codegen id : 6] +(15) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 -(15) Scan parquet default.web_returns +(16) Scan parquet default.web_returns Output [1]: [wr_order_number#13] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/web_returns] +PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 7] +(17) ColumnarToRow [codegen id : 7] +Input [1]: [wr_order_number#13] + +(18) Filter [codegen id : 7] Input [1]: [wr_order_number#13] +Condition : isnotnull(wr_order_number#13) -(17) Exchange +(19) Exchange Input [1]: [wr_order_number#13] Arguments: hashpartitioning(wr_order_number#13, 5), true, [id=#14] -(18) Sort [codegen id : 8] +(20) Sort [codegen id : 8] Input [1]: [wr_order_number#13] Arguments: [wr_order_number#13 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin +(21) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] Right keys [1]: [wr_order_number#13] Join condition: None -(20) Scan parquet default.customer_address +(22) Scan parquet default.customer_address Output [2]: [ca_address_sk#15, ca_state#16] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(21) ColumnarToRow [codegen id : 9] +(23) ColumnarToRow [codegen id : 9] Input [2]: [ca_address_sk#15, ca_state#16] -(22) Filter [codegen id : 9] +(24) Filter [codegen id : 9] Input [2]: [ca_address_sk#15, ca_state#16] Condition : ((isnotnull(ca_state#16) AND (ca_state#16 = IL)) AND isnotnull(ca_address_sk#15)) -(23) Project [codegen id : 9] +(25) Project [codegen id : 9] Output [1]: [ca_address_sk#15] Input [2]: [ca_address_sk#15, ca_state#16] -(24) BroadcastExchange +(26) BroadcastExchange Input [1]: [ca_address_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] -(25) BroadcastHashJoin [codegen id : 12] +(27) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#15] Join condition: None -(26) Project [codegen id : 12] +(28) Project [codegen id : 12] Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#15] -(27) Scan parquet default.web_site +(29) Scan parquet default.web_site Output [2]: [web_site_sk#18, web_company_name#19] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 10] +(30) ColumnarToRow [codegen id : 10] Input [2]: [web_site_sk#18, web_company_name#19] -(29) Filter [codegen id : 10] +(31) Filter [codegen id : 10] Input [2]: [web_site_sk#18, web_company_name#19] Condition : ((isnotnull(web_company_name#19) AND (web_company_name#19 = pri)) AND isnotnull(web_site_sk#18)) -(30) Project [codegen id : 10] +(32) Project [codegen id : 10] Output [1]: [web_site_sk#18] Input [2]: [web_site_sk#18, web_company_name#19] -(31) BroadcastExchange +(33) BroadcastExchange Input [1]: [web_site_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] -(32) BroadcastHashJoin [codegen id : 12] +(34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#18] Join condition: None -(33) Project [codegen id : 12] +(35) Project [codegen id : 12] Output [4]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -(34) Scan parquet default.date_dim +(36) Scan parquet default.date_dim Output [2]: [d_date_sk#21, d_date#22] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 11] +(37) ColumnarToRow [codegen id : 11] Input [2]: [d_date_sk#21, d_date#22] -(36) Filter [codegen id : 11] +(38) Filter [codegen id : 11] Input [2]: [d_date_sk#21, d_date#22] Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 10623)) AND (d_date#22 <= 10683)) AND isnotnull(d_date_sk#21)) -(37) Project [codegen id : 11] +(39) Project [codegen id : 11] Output [1]: [d_date_sk#21] Input [2]: [d_date_sk#21, d_date#22] -(38) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] -(39) BroadcastHashJoin [codegen id : 12] +(41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#21] Join condition: None -(40) Project [codegen id : 12] +(42) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#21] -(41) HashAggregate [codegen id : 12] +(43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#24, sum(UnscaledValue(ws_net_profit#7))#25] Results [3]: [ws_order_number#5, sum#26, sum#27] -(42) Exchange +(44) Exchange Input [3]: [ws_order_number#5, sum#26, sum#27] Arguments: hashpartitioning(ws_order_number#5, 5), true, [id=#28] -(43) HashAggregate [codegen id : 13] +(45) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#26, sum#27] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#24, sum(UnscaledValue(ws_net_profit#7))#25] Results [3]: [ws_order_number#5, sum#26, sum#27] -(44) HashAggregate [codegen id : 13] +(46) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#26, sum#27] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#24, sum(UnscaledValue(ws_net_profit#7))#25, count(ws_order_number#5)#29] Results [3]: [sum#26, sum#27, count#30] -(45) Exchange +(47) Exchange Input [3]: [sum#26, sum#27, count#30] Arguments: SinglePartition, true, [id=#31] -(46) HashAggregate [codegen id : 14] +(48) HashAggregate [codegen id : 14] Input [3]: [sum#26, sum#27, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#24, sum(UnscaledValue(ws_net_profit#7))#25, count(ws_order_number#5)#29] Results [3]: [count(ws_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#24,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#25,17,2) AS total net profit #34] -(47) TakeOrderedAndProject +(49) TakeOrderedAndProject Input [3]: [order count #32, total shipping cost #33, total net profit #34] Arguments: 100, [order count #32 ASC NULLS FIRST], [order count #32, total shipping cost #33, total net profit #34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt index 95ff0dbb4ad50..7b2c6568adfad 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt @@ -31,7 +31,7 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] InputAdapter Exchange [ws_order_number] #4 WholeStageCodegen (1) - Filter [ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] + Filter [ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_warehouse_sk,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_warehouse_sk,ws_web_site_sk] @@ -41,17 +41,19 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Exchange [ws_order_number] #5 WholeStageCodegen (3) Project [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_order_number,ws_warehouse_sk] + Filter [ws_order_number,ws_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_order_number,ws_warehouse_sk] WholeStageCodegen (8) Sort [wr_order_number] InputAdapter Exchange [wr_order_number] #6 WholeStageCodegen (7) - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number] + Filter [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number] InputAdapter BroadcastExchange #7 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 3a7c91dc09301..b5ac520a8b63e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -1,52 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * HashAggregate (40) - +- Exchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * BroadcastHashJoin LeftAnti BuildRight (13) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (8) +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (29) + : +- * BroadcastHashJoin Inner BuildRight (28) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * BroadcastHashJoin LeftAnti BuildRight (15) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (9) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Project (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.web_sales (4) - : : : +- BroadcastExchange (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.web_returns (10) - : : +- BroadcastExchange (18) - : : +- * Project (17) - : : +- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet default.date_dim (14) - : +- BroadcastExchange (25) - : +- * Project (24) - : +- * Filter (23) - : +- * ColumnarToRow (22) - : +- Scan parquet default.customer_address (21) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- * ColumnarToRow (29) - +- Scan parquet default.web_site (28) + : : : : +- BroadcastExchange (8) + : : : : +- * Project (7) + : : : : +- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.web_sales (4) + : : : +- BroadcastExchange (14) + : : : +- * Filter (13) + : : : +- * ColumnarToRow (12) + : : : +- Scan parquet default.web_returns (11) + : : +- BroadcastExchange (20) + : : +- * Project (19) + : : +- * Filter (18) + : : +- * ColumnarToRow (17) + : : +- Scan parquet default.date_dim (16) + : +- BroadcastExchange (27) + : +- * Project (26) + : +- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.customer_address (23) + +- BroadcastExchange (34) + +- * Project (33) + +- * Filter (32) + +- * ColumnarToRow (31) + +- Scan parquet default.web_site (30) (1) Scan parquet default.web_sales Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 6] @@ -54,182 +56,192 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (3) Filter [codegen id : 6] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) +Condition : ((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#5)) AND isnotnull(ws_warehouse_sk#4)) (4) Scan parquet default.web_sales Output [2]: [ws_warehouse_sk#4, ws_order_number#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/web_sales] +PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] Input [2]: [ws_warehouse_sk#4, ws_order_number#5] -(6) Project [codegen id : 1] +(6) Filter [codegen id : 1] +Input [2]: [ws_warehouse_sk#4, ws_order_number#5] +Condition : (isnotnull(ws_order_number#5) AND isnotnull(ws_warehouse_sk#4)) + +(7) Project [codegen id : 1] Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#8, ws_order_number#5 AS ws_order_number#5#9] Input [2]: [ws_warehouse_sk#4, ws_order_number#5] -(7) BroadcastExchange +(8) BroadcastExchange Input [2]: [ws_warehouse_sk#4#8, ws_order_number#5#9] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#10] -(8) BroadcastHashJoin [codegen id : 6] +(9) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#5#9] Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#8) -(9) Project [codegen id : 6] +(10) Project [codegen id : 6] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(10) Scan parquet default.web_returns +(11) Scan parquet default.web_returns Output [1]: [wr_order_number#11] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/web_returns] +PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 2] +(12) ColumnarToRow [codegen id : 2] +Input [1]: [wr_order_number#11] + +(13) Filter [codegen id : 2] Input [1]: [wr_order_number#11] +Condition : isnotnull(wr_order_number#11) -(12) BroadcastExchange +(14) BroadcastExchange Input [1]: [wr_order_number#11] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [id=#12] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [id=#12] -(13) BroadcastHashJoin [codegen id : 6] +(15) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cast(ws_order_number#5 as bigint)] Right keys [1]: [wr_order_number#11] Join condition: None -(14) Scan parquet default.date_dim +(16) Scan parquet default.date_dim Output [2]: [d_date_sk#13, d_date#14] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 3] +(17) ColumnarToRow [codegen id : 3] Input [2]: [d_date_sk#13, d_date#14] -(16) Filter [codegen id : 3] +(18) Filter [codegen id : 3] Input [2]: [d_date_sk#13, d_date#14] Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 10623)) AND (d_date#14 <= 10683)) AND isnotnull(d_date_sk#13)) -(17) Project [codegen id : 3] +(19) Project [codegen id : 3] Output [1]: [d_date_sk#13] Input [2]: [d_date_sk#13, d_date#14] -(18) BroadcastExchange +(20) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] -(19) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#13] Join condition: None -(20) Project [codegen id : 6] +(22) Project [codegen id : 6] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#13] -(21) Scan parquet default.customer_address +(23) Scan parquet default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(22) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 4] Input [2]: [ca_address_sk#16, ca_state#17] -(23) Filter [codegen id : 4] +(25) Filter [codegen id : 4] Input [2]: [ca_address_sk#16, ca_state#17] Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = IL)) AND isnotnull(ca_address_sk#16)) -(24) Project [codegen id : 4] +(26) Project [codegen id : 4] Output [1]: [ca_address_sk#16] Input [2]: [ca_address_sk#16, ca_state#17] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(26) BroadcastHashJoin [codegen id : 6] +(28) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join condition: None -(27) Project [codegen id : 6] +(29) Project [codegen id : 6] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] -(28) Scan parquet default.web_site +(30) Scan parquet default.web_site Output [2]: [web_site_sk#19, web_company_name#20] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 5] +(31) ColumnarToRow [codegen id : 5] Input [2]: [web_site_sk#19, web_company_name#20] -(30) Filter [codegen id : 5] +(32) Filter [codegen id : 5] Input [2]: [web_site_sk#19, web_company_name#20] Condition : ((isnotnull(web_company_name#20) AND (web_company_name#20 = pri)) AND isnotnull(web_site_sk#19)) -(31) Project [codegen id : 5] +(33) Project [codegen id : 5] Output [1]: [web_site_sk#19] Input [2]: [web_site_sk#19, web_company_name#20] -(32) BroadcastExchange +(34) BroadcastExchange Input [1]: [web_site_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(33) BroadcastHashJoin [codegen id : 6] +(35) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#19] Join condition: None -(34) Project [codegen id : 6] +(36) Project [codegen id : 6] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#19] -(35) HashAggregate [codegen id : 6] +(37) HashAggregate [codegen id : 6] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23] Results [3]: [ws_order_number#5, sum#24, sum#25] -(36) Exchange +(38) Exchange Input [3]: [ws_order_number#5, sum#24, sum#25] Arguments: hashpartitioning(ws_order_number#5, 5), true, [id=#26] -(37) HashAggregate [codegen id : 7] +(39) HashAggregate [codegen id : 7] Input [3]: [ws_order_number#5, sum#24, sum#25] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23] Results [3]: [ws_order_number#5, sum#24, sum#25] -(38) HashAggregate [codegen id : 7] +(40) HashAggregate [codegen id : 7] Input [3]: [ws_order_number#5, sum#24, sum#25] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#27] Results [3]: [sum#24, sum#25, count#28] -(39) Exchange +(41) Exchange Input [3]: [sum#24, sum#25, count#28] Arguments: SinglePartition, true, [id=#29] -(40) HashAggregate [codegen id : 8] +(42) HashAggregate [codegen id : 8] Input [3]: [sum#24, sum#25, count#28] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#27] Results [3]: [count(ws_order_number#5)#27 AS order count #30, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#22,17,2) AS total shipping cost #31, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#23,17,2) AS total net profit #32] -(41) TakeOrderedAndProject +(43) TakeOrderedAndProject Input [3]: [order count #30, total shipping cost #31, total net profit #32] Arguments: 100, [order count #30 ASC NULLS FIRST], [order count #30, total shipping cost #31, total net profit #32] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index fda5a9f037ad4..4b086d9a90f34 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] BroadcastHashJoin [wr_order_number,ws_order_number] Project [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] BroadcastHashJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - Filter [ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] + Filter [ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_warehouse_sk,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_warehouse_sk,ws_web_site_sk] @@ -27,15 +27,17 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] BroadcastExchange #3 WholeStageCodegen (1) Project [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_order_number,ws_warehouse_sk] + Filter [ws_order_number,ws_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_order_number,ws_warehouse_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number] + Filter [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt index c2baa5ba23ffc..7e91549c1894d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt @@ -68,7 +68,7 @@ TakeOrderedAndProject (63) Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -76,7 +76,7 @@ Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num (3) Filter [codegen id : 1] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) +Condition : (((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#4)) (4) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt index 420cd6f89b02a..b7787108bebd6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] InputAdapter Exchange [ws_order_number] #4 WholeStageCodegen (1) - Filter [ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] + Filter [ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index 5b232d915efdd..c6ddcce9abf26 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -61,7 +61,7 @@ TakeOrderedAndProject (56) Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -69,7 +69,7 @@ Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num (3) Filter [codegen id : 9] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) +Condition : (((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#4)) (4) Scan parquet default.web_sales Output [2]: [ws_warehouse_sk#7, ws_order_number#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 00f1e8cc15922..b5470d530a411 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] BroadcastHashJoin [d_date_sk,ws_ship_date_sk] BroadcastHashJoin [wr_order_number,ws_order_number] BroadcastHashJoin [ws_order_number,ws_order_number] - Filter [ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] + Filter [ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index 26797aa2de40e..887a4f4644878 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -57,7 +57,7 @@ TakeOrderedAndProject (52) Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -65,7 +65,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -79,7 +79,7 @@ Arguments: [c_customer_sk#1 ASC NULLS FIRST], false, 0 Output [2]: [ss_sold_date_sk#5, ss_customer_sk#6] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] @@ -87,7 +87,7 @@ Input [2]: [ss_sold_date_sk#5, ss_customer_sk#6] (8) Filter [codegen id : 4] Input [2]: [ss_sold_date_sk#5, ss_customer_sk#6] -Condition : isnotnull(ss_sold_date_sk#5) +Condition : (isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_customer_sk#6)) (9) Scan parquet default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_moy#9] @@ -137,7 +137,7 @@ Join condition: None Output [2]: [ws_sold_date_sk#12, ws_bill_customer_sk#13] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (20) ColumnarToRow [codegen id : 7] @@ -145,7 +145,7 @@ Input [2]: [ws_sold_date_sk#12, ws_bill_customer_sk#13] (21) Filter [codegen id : 7] Input [2]: [ws_sold_date_sk#12, ws_bill_customer_sk#13] -Condition : isnotnull(ws_sold_date_sk#12) +Condition : (isnotnull(ws_sold_date_sk#12) AND isnotnull(ws_bill_customer_sk#13)) (22) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#7] @@ -163,7 +163,7 @@ Input [3]: [ws_sold_date_sk#12, ws_bill_customer_sk#13, d_date_sk#7] Output [2]: [cs_sold_date_sk#15, cs_ship_customer_sk#16] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 9] @@ -171,7 +171,7 @@ Input [2]: [cs_sold_date_sk#15, cs_ship_customer_sk#16] (27) Filter [codegen id : 9] Input [2]: [cs_sold_date_sk#15, cs_ship_customer_sk#16] -Condition : isnotnull(cs_sold_date_sk#15) +Condition : (isnotnull(cs_sold_date_sk#15) AND isnotnull(cs_ship_customer_sk#16)) (28) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt index 9a144aee5f12d..828e7cb4081e4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] @@ -32,7 +32,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] @@ -52,7 +52,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep WholeStageCodegen (7) Project [ws_bill_customer_sk] BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_sold_date_sk] + Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] @@ -61,7 +61,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep WholeStageCodegen (9) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk] + Filter [cs_ship_customer_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index aba866b2117a1..a91bf42b0edf5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -53,7 +53,7 @@ TakeOrderedAndProject (48) Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -61,13 +61,13 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#4, ss_customer_sk#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] @@ -75,7 +75,7 @@ Input [2]: [ss_sold_date_sk#4, ss_customer_sk#5] (6) Filter [codegen id : 2] Input [2]: [ss_sold_date_sk#4, ss_customer_sk#5] -Condition : isnotnull(ss_sold_date_sk#4) +Condition : (isnotnull(ss_sold_date_sk#4) AND isnotnull(ss_customer_sk#5)) (7) Scan parquet default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] @@ -121,7 +121,7 @@ Join condition: None Output [2]: [ws_sold_date_sk#11, ws_bill_customer_sk#12] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 4] @@ -129,7 +129,7 @@ Input [2]: [ws_sold_date_sk#11, ws_bill_customer_sk#12] (18) Filter [codegen id : 4] Input [2]: [ws_sold_date_sk#11, ws_bill_customer_sk#12] -Condition : isnotnull(ws_sold_date_sk#11) +Condition : (isnotnull(ws_sold_date_sk#11) AND isnotnull(ws_bill_customer_sk#12)) (19) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#6] @@ -147,7 +147,7 @@ Input [3]: [ws_sold_date_sk#11, ws_bill_customer_sk#12, d_date_sk#6] Output [2]: [cs_sold_date_sk#14, cs_ship_customer_sk#15] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_customer_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -155,7 +155,7 @@ Input [2]: [cs_sold_date_sk#14, cs_ship_customer_sk#15] (24) Filter [codegen id : 6] Input [2]: [cs_sold_date_sk#14, cs_ship_customer_sk#15] -Condition : isnotnull(cs_sold_date_sk#14) +Condition : (isnotnull(cs_sold_date_sk#14) AND isnotnull(cs_ship_customer_sk#15)) (25) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 90105ffd73ea8..b3bbbcb360422 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Project [c_current_addr_sk,c_current_cdemo_sk] BroadcastHashJoin [c_customer_sk,customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] @@ -21,7 +21,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] @@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_sold_date_sk] + Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] @@ -48,7 +48,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk] + Filter [cs_ship_customer_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] 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 5282470abdc5f..51e819697ff56 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 @@ -137,7 +137,7 @@ Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_class_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_class_id), IsNotNull(i_item_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 17] @@ -145,7 +145,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (8) 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_category_id#9)) AND isnotnull(i_class_id#8)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_category_id#9)) AND isnotnull(i_class_id#8)) AND isnotnull(i_item_sk#6)) (9) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] 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 5141d40e7325b..ee77670ddc6b3 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 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat WholeStageCodegen (17) Project [i_item_sk] BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 9ee58f9b3d604..f7bb2844f18ea 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -119,7 +119,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/item] -PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_brand_id)] +PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_brand_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -127,7 +127,7 @@ Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : ((isnotnull(i_class_id#7) AND isnotnull(i_category_id#8)) AND isnotnull(i_brand_id#6)) +Condition : (((isnotnull(i_class_id#7) AND isnotnull(i_category_id#8)) AND isnotnull(i_brand_id#6)) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index f1668ea399807..7892b6727f9a3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] 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 705abacb4f572..0187c02cbecc8 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 @@ -249,7 +249,7 @@ Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_class_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_class_id), IsNotNull(i_item_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 17] @@ -257,7 +257,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (8) 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_category_id#9)) AND isnotnull(i_class_id#8)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_category_id#9)) AND isnotnull(i_class_id#8)) AND isnotnull(i_item_sk#6)) (9) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] 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 7a78f19e5cc99..7788bce034636 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 @@ -117,7 +117,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,number_sales, WholeStageCodegen (17) Project [i_item_sk] BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index 24f9a69287dc9..8c3627a73fdb9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -225,7 +225,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/item] -PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_brand_id)] +PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_brand_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -233,7 +233,7 @@ Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : ((isnotnull(i_class_id#7) AND isnotnull(i_category_id#8)) AND isnotnull(i_brand_id#6)) +Condition : (((isnotnull(i_class_id#7) AND isnotnull(i_category_id#8)) AND isnotnull(i_brand_id#6)) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index a329e40a70bbe..213437e439b4d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,number_sales, WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt index 25b48af7f658f..bb534e48f20fa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt @@ -65,7 +65,7 @@ TakeOrderedAndProject (60) Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -73,7 +73,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -87,7 +87,7 @@ Arguments: [c_customer_sk#3 ASC NULLS FIRST], false, 0 Output [2]: [ss_sold_date_sk#7, ss_customer_sk#8] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] @@ -95,7 +95,7 @@ Input [2]: [ss_sold_date_sk#7, ss_customer_sk#8] (8) Filter [codegen id : 4] Input [2]: [ss_sold_date_sk#7, ss_customer_sk#8] -Condition : isnotnull(ss_sold_date_sk#7) +Condition : (isnotnull(ss_sold_date_sk#7) AND isnotnull(ss_customer_sk#8)) (9) Scan parquet default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt index 3e21f6dae18d6..0be550ebfbc7b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] @@ -43,7 +43,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index a6341c55f0457..c3c1d644bed98 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -54,7 +54,7 @@ TakeOrderedAndProject (49) Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -62,13 +62,13 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#6, ss_customer_sk#7] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] @@ -76,7 +76,7 @@ Input [2]: [ss_sold_date_sk#6, ss_customer_sk#7] (6) Filter [codegen id : 2] Input [2]: [ss_sold_date_sk#6, ss_customer_sk#7] -Condition : isnotnull(ss_sold_date_sk#6) +Condition : (isnotnull(ss_sold_date_sk#6) AND isnotnull(ss_customer_sk#7)) (7) Scan parquet default.date_dim Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index 2da6d615fe17d..a674baaa478a5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] @@ -23,7 +23,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt index 9e668c7015769..0018ad94d9bc5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt @@ -62,7 +62,7 @@ TakeOrderedAndProject (57) Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -70,7 +70,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -84,7 +84,7 @@ Arguments: [c_customer_sk#1 ASC NULLS FIRST], false, 0 Output [2]: [ss_sold_date_sk#5, ss_customer_sk#6] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] @@ -92,7 +92,7 @@ Input [2]: [ss_sold_date_sk#5, ss_customer_sk#6] (8) Filter [codegen id : 4] Input [2]: [ss_sold_date_sk#5, ss_customer_sk#6] -Condition : isnotnull(ss_sold_date_sk#5) +Condition : (isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_customer_sk#6)) (9) Scan parquet default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] @@ -142,7 +142,7 @@ Join condition: None Output [2]: [ws_sold_date_sk#12, ws_bill_customer_sk#13] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (20) ColumnarToRow [codegen id : 7] @@ -150,7 +150,7 @@ Input [2]: [ws_sold_date_sk#12, ws_bill_customer_sk#13] (21) Filter [codegen id : 7] Input [2]: [ws_sold_date_sk#12, ws_bill_customer_sk#13] -Condition : isnotnull(ws_sold_date_sk#12) +Condition : (isnotnull(ws_sold_date_sk#12) AND isnotnull(ws_bill_customer_sk#13)) (22) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#7] @@ -168,7 +168,7 @@ Input [3]: [ws_sold_date_sk#12, ws_bill_customer_sk#13, d_date_sk#7] Output [2]: [cs_sold_date_sk#15, cs_ship_customer_sk#16] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 9] @@ -176,7 +176,7 @@ Input [2]: [cs_sold_date_sk#15, cs_ship_customer_sk#16] (27) Filter [codegen id : 9] Input [2]: [cs_sold_date_sk#15, cs_ship_customer_sk#16] -Condition : isnotnull(cs_sold_date_sk#15) +Condition : (isnotnull(cs_sold_date_sk#15) AND isnotnull(cs_ship_customer_sk#16)) (28) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt index 2840f275ff2b7..92541f5d2c17c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt @@ -30,7 +30,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] @@ -41,7 +41,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] @@ -61,7 +61,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em WholeStageCodegen (7) Project [ws_bill_customer_sk] BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_sold_date_sk] + Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] @@ -70,7 +70,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em WholeStageCodegen (9) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk] + Filter [cs_ship_customer_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index 69ad4ba7290bf..6a97f16f4f2e8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -52,7 +52,7 @@ TakeOrderedAndProject (47) Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -60,13 +60,13 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#4, ss_customer_sk#5] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] @@ -74,7 +74,7 @@ Input [2]: [ss_sold_date_sk#4, ss_customer_sk#5] (6) Filter [codegen id : 2] Input [2]: [ss_sold_date_sk#4, ss_customer_sk#5] -Condition : isnotnull(ss_sold_date_sk#4) +Condition : (isnotnull(ss_sold_date_sk#4) AND isnotnull(ss_customer_sk#5)) (7) Scan parquet default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_qoy#8] @@ -120,7 +120,7 @@ Join condition: None Output [2]: [ws_sold_date_sk#11, ws_bill_customer_sk#12] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 4] @@ -128,7 +128,7 @@ Input [2]: [ws_sold_date_sk#11, ws_bill_customer_sk#12] (18) Filter [codegen id : 4] Input [2]: [ws_sold_date_sk#11, ws_bill_customer_sk#12] -Condition : isnotnull(ws_sold_date_sk#11) +Condition : (isnotnull(ws_sold_date_sk#11) AND isnotnull(ws_bill_customer_sk#12)) (19) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#6] @@ -146,7 +146,7 @@ Input [3]: [ws_sold_date_sk#11, ws_bill_customer_sk#12, d_date_sk#6] Output [2]: [cs_sold_date_sk#14, cs_ship_customer_sk#15] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_customer_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -154,7 +154,7 @@ Input [2]: [cs_sold_date_sk#14, cs_ship_customer_sk#15] (24) Filter [codegen id : 6] Input [2]: [cs_sold_date_sk#14, cs_ship_customer_sk#15] -Condition : isnotnull(cs_sold_date_sk#14) +Condition : (isnotnull(cs_sold_date_sk#14) AND isnotnull(cs_ship_customer_sk#15)) (25) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index 054296501acf2..a39818d9aa65e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em Project [c_current_addr_sk,c_current_cdemo_sk] BroadcastHashJoin [c_customer_sk,customsk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] @@ -21,7 +21,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] @@ -39,7 +39,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_sold_date_sk] + Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] @@ -48,7 +48,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk] + Filter [cs_ship_customer_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt index 77a7e2a00c4f4..4ff09975eb059 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt @@ -103,6 +103,7 @@ TakeOrderedAndProject (98) Output [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -110,7 +111,7 @@ Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profi (3) Filter [codegen id : 1] Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Condition : (isnotnull(cast(ss_sold_date_sk#1 as bigint)) AND isnotnull(cast(ss_store_sk#2 as bigint))) +Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#2)) AND isnotnull(cast(ss_sold_date_sk#1 as bigint))) AND isnotnull(cast(ss_store_sk#2 as bigint))) (4) Project [codegen id : 1] Output [6]: [cast(ss_store_sk#2 as bigint) AS store_sk#5, cast(ss_sold_date_sk#1 as bigint) AS date_sk#6, ss_ext_sales_price#3 AS sales_price#7, ss_net_profit#4 AS profit#8, 0.00 AS return_amt#9, 0.00 AS net_loss#10] @@ -311,7 +312,7 @@ Results [5]: [catalog channel AS channel#81, concat(catalog_page, cp_catalog_pag Output [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/web_sales] -PushedFilters: [IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] @@ -319,7 +320,7 @@ Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net (49) Filter [codegen id : 13] Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] -Condition : (isnotnull(cast(ws_sold_date_sk#86 as bigint)) AND isnotnull(ws_web_site_sk#87)) +Condition : ((isnotnull(ws_sold_date_sk#86) AND isnotnull(cast(ws_sold_date_sk#86 as bigint))) AND isnotnull(ws_web_site_sk#87)) (50) Project [codegen id : 13] Output [6]: [ws_web_site_sk#87 AS wsr_web_site_sk#90, cast(ws_sold_date_sk#86 as bigint) AS date_sk#91, ws_ext_sales_price#88 AS sales_price#92, ws_net_profit#89 AS profit#93, 0.00 AS return_amt#94, 0.00 AS net_loss#95] @@ -329,7 +330,7 @@ Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net Output [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/web_returns] -PushedFilters: [IsNotNull(wr_returned_date_sk)] +PushedFilters: [IsNotNull(wr_returned_date_sk), IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] @@ -337,7 +338,7 @@ Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return (53) Filter [codegen id : 14] Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] -Condition : isnotnull(wr_returned_date_sk#96) +Condition : ((isnotnull(wr_returned_date_sk#96) AND isnotnull(wr_item_sk#97)) AND isnotnull(wr_order_number#98)) (54) Exchange Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt index 209f65e8d333e..12e3c2ed8403c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt @@ -122,7 +122,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] InputAdapter Exchange [wr_item_sk,wr_order_number] #10 WholeStageCodegen (14) - Filter [wr_returned_date_sk] + Filter [wr_item_sk,wr_order_number,wr_returned_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_net_loss,wr_order_number,wr_return_amt,wr_returned_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 62bbb6547080a..3566750e43404 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -100,6 +100,7 @@ TakeOrderedAndProject (95) Output [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -107,7 +108,7 @@ Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profi (3) Filter [codegen id : 1] Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Condition : (isnotnull(cast(ss_sold_date_sk#1 as bigint)) AND isnotnull(cast(ss_store_sk#2 as bigint))) +Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_store_sk#2)) AND isnotnull(cast(ss_sold_date_sk#1 as bigint))) AND isnotnull(cast(ss_store_sk#2 as bigint))) (4) Project [codegen id : 1] Output [6]: [cast(ss_store_sk#2 as bigint) AS store_sk#5, cast(ss_sold_date_sk#1 as bigint) AS date_sk#6, ss_ext_sales_price#3 AS sales_price#7, ss_net_profit#4 AS profit#8, 0.00 AS return_amt#9, 0.00 AS net_loss#10] @@ -308,7 +309,7 @@ Results [5]: [catalog channel AS channel#81, concat(catalog_page, cp_catalog_pag Output [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/web_sales] -PushedFilters: [IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] @@ -316,7 +317,7 @@ Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net (49) Filter [codegen id : 13] Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] -Condition : (isnotnull(cast(ws_sold_date_sk#86 as bigint)) AND isnotnull(ws_web_site_sk#87)) +Condition : ((isnotnull(ws_sold_date_sk#86) AND isnotnull(cast(ws_sold_date_sk#86 as bigint))) AND isnotnull(ws_web_site_sk#87)) (50) Project [codegen id : 13] Output [6]: [ws_web_site_sk#87 AS wsr_web_site_sk#90, cast(ws_sold_date_sk#86 as bigint) AS date_sk#91, ws_ext_sales_price#88 AS sales_price#92, ws_net_profit#89 AS profit#93, 0.00 AS return_amt#94, 0.00 AS net_loss#95] @@ -326,7 +327,7 @@ Input [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net Output [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/web_returns] -PushedFilters: [IsNotNull(wr_returned_date_sk)] +PushedFilters: [IsNotNull(wr_returned_date_sk), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 15] @@ -334,7 +335,7 @@ Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return (53) Filter [codegen id : 15] Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] -Condition : isnotnull(wr_returned_date_sk#96) +Condition : ((isnotnull(wr_returned_date_sk#96) AND isnotnull(wr_order_number#98)) AND isnotnull(wr_item_sk#97)) (54) Scan parquet default.web_sales Output [3]: [ws_item_sk#101, ws_web_site_sk#87, ws_order_number#102] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index 77d7d6f938665..deb094d75b305 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -116,7 +116,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] WholeStageCodegen (15) Project [wr_net_loss,wr_return_amt,wr_returned_date_sk,ws_web_site_sk] BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Filter [wr_returned_date_sk] + Filter [wr_item_sk,wr_order_number,wr_returned_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_net_loss,wr_order_number,wr_return_amt,wr_returned_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt index 8ffe0713c70ae..aac3c6b05d054 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt @@ -114,7 +114,7 @@ Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3, d_date_sk#4] Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilityWithStatsSuite/store] -PushedFilters: [IsNotNull(s_store_sk)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -122,7 +122,7 @@ Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) +Condition : (isnotnull(s_store_sk#7) AND isnotnull(s_state#9)) (14) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3] @@ -209,7 +209,7 @@ Arguments: [rank(_w2#16) windowspecdefinition(s_state#9, _w2#16 DESC NULLS LAST, (32) Filter [codegen id : 7] Input [4]: [s_state#15, s_state#9, _w2#16, ranking#18] -Condition : (isnotnull(ranking#18) AND (ranking#18 <= 5)) +Condition : ((isnotnull(ranking#18) AND (ranking#18 <= 5)) AND isnotnull(s_state#15)) (33) Project [codegen id : 7] Output [1]: [s_state#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt index f99d2b02f8429..b7ae9cc169011 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt @@ -49,7 +49,7 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su BroadcastExchange #6 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk] + Filter [s_state,s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_county,s_state,s_store_sk] @@ -57,7 +57,7 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su BroadcastExchange #7 WholeStageCodegen (7) Project [s_state] - Filter [ranking] + Filter [ranking,s_state] InputAdapter Window [_w2,s_state] WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index 6fc2c3e2b48a1..3e2978c7ee8ec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -114,7 +114,7 @@ Input [4]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3, d_date_sk#4] Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV2_7_PlanStabilitySuite/store] -PushedFilters: [IsNotNull(s_store_sk)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -122,7 +122,7 @@ Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) +Condition : (isnotnull(s_store_sk#7) AND isnotnull(s_state#9)) (14) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_store_sk#2, ss_net_profit#3] @@ -209,7 +209,7 @@ Arguments: [rank(_w2#16) windowspecdefinition(s_state#9, _w2#16 DESC NULLS LAST, (32) Filter [codegen id : 7] Input [4]: [s_state#15, s_state#9, _w2#16, ranking#18] -Condition : (isnotnull(ranking#18) AND (ranking#18 <= 5)) +Condition : ((isnotnull(ranking#18) AND (ranking#18 <= 5)) AND isnotnull(s_state#15)) (33) Project [codegen id : 7] Output [1]: [s_state#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 1cf0fbbea7ac6..7bf6770f4bb98 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -49,7 +49,7 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su BroadcastExchange #6 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk] + Filter [s_state,s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_county,s_state,s_store_sk] @@ -57,7 +57,7 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su BroadcastExchange #7 WholeStageCodegen (7) Project [s_state] - Filter [ranking] + Filter [ranking,s_state] InputAdapter Window [_w2,s_state] WholeStageCodegen (6) From 9e9a633b902e465ac87549352417067553b7edc2 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 12 Feb 2021 22:50:58 +0800 Subject: [PATCH 7/9] Infer filters from DPP --- .../sql/catalyst/optimizer/Optimizer.scala | 56 +- .../plans/logical/QueryPlanConstraints.scala | 10 +- .../spark/sql/execution/SparkOptimizer.scala | 5 +- .../CleanupDynamicPruningFilters.scala | 27 +- .../InferDynamicPruningFilters.scala | 41 ++ .../dynamicpruning/PartitionPruning.scala | 4 +- .../approved-plans-v1_4/q10.sf100/explain.txt | 214 +++---- .../q10.sf100/simplified.txt | 13 +- .../approved-plans-v1_4/q10/explain.txt | 190 ++++--- .../approved-plans-v1_4/q10/simplified.txt | 13 +- .../q14a.sf100/explain.txt | 4 +- .../q14a.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q14a/explain.txt | 4 +- .../approved-plans-v1_4/q14a/simplified.txt | 2 +- .../q14b.sf100/explain.txt | 4 +- .../q14b.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q14b/explain.txt | 4 +- .../approved-plans-v1_4/q14b/simplified.txt | 2 +- .../approved-plans-v1_4/q16.sf100/explain.txt | 174 +++--- .../q16.sf100/simplified.txt | 16 +- .../approved-plans-v1_4/q16/explain.txt | 174 +++--- .../approved-plans-v1_4/q16/simplified.txt | 16 +- .../q23a.sf100/explain.txt | 522 +++++++++--------- .../q23a.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q23a/explain.txt | 418 +++++++------- .../approved-plans-v1_4/q23a/simplified.txt | 20 +- .../q23b.sf100/explain.txt | 8 +- .../q23b.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q23b/explain.txt | 8 +- .../approved-plans-v1_4/q23b/simplified.txt | 4 +- .../approved-plans-v1_4/q33.sf100/explain.txt | 8 +- .../q33.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q33/explain.txt | 8 +- .../approved-plans-v1_4/q35.sf100/explain.txt | 222 ++++---- .../q35.sf100/simplified.txt | 13 +- .../approved-plans-v1_4/q35/explain.txt | 186 ++++--- .../approved-plans-v1_4/q35/simplified.txt | 13 +- .../approved-plans-v1_4/q5.sf100/explain.txt | 136 ++--- .../q5.sf100/simplified.txt | 9 +- .../approved-plans-v1_4/q5/explain.txt | 133 ++--- .../approved-plans-v1_4/q5/simplified.txt | 9 +- .../approved-plans-v1_4/q56.sf100/explain.txt | 8 +- .../q56.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q56/explain.txt | 8 +- .../approved-plans-v1_4/q56/simplified.txt | 2 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 367 ++++++------ .../q58.sf100/simplified.txt | 36 +- .../approved-plans-v1_4/q58/explain.txt | 369 +++++++------ .../approved-plans-v1_4/q58/simplified.txt | 44 +- .../approved-plans-v1_4/q60.sf100/explain.txt | 8 +- .../q60.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q60/explain.txt | 8 +- .../approved-plans-v1_4/q60/simplified.txt | 2 +- .../approved-plans-v1_4/q69.sf100/explain.txt | 210 +++---- .../q69.sf100/simplified.txt | 31 +- .../approved-plans-v1_4/q69/explain.txt | 198 ++++--- .../approved-plans-v1_4/q69/simplified.txt | 31 +- .../approved-plans-v1_4/q70.sf100/explain.txt | 6 +- .../q70.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q70/explain.txt | 6 +- .../approved-plans-v1_4/q70/simplified.txt | 2 +- .../approved-plans-v1_4/q83.sf100/explain.txt | 496 +++++++++-------- .../q83.sf100/simplified.txt | 9 +- .../approved-plans-v1_4/q83/explain.txt | 474 ++++++++-------- .../approved-plans-v1_4/q83/simplified.txt | 21 +- .../approved-plans-v1_4/q93.sf100/explain.txt | 50 +- .../q93.sf100/simplified.txt | 7 +- .../approved-plans-v1_4/q93/explain.txt | 102 ++-- .../approved-plans-v1_4/q93/simplified.txt | 7 +- .../approved-plans-v1_4/q94.sf100/explain.txt | 190 ++++--- .../q94.sf100/simplified.txt | 16 +- .../approved-plans-v1_4/q94/explain.txt | 190 ++++--- .../approved-plans-v1_4/q94/simplified.txt | 16 +- .../approved-plans-v1_4/q95.sf100/explain.txt | 10 +- .../q95.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q95/explain.txt | 10 +- .../approved-plans-v1_4/q95/simplified.txt | 2 +- .../q10a.sf100/explain.txt | 202 ++++--- .../q10a.sf100/simplified.txt | 31 +- .../approved-plans-v2_7/q10a/explain.txt | 194 ++++--- .../approved-plans-v2_7/q10a/simplified.txt | 31 +- .../approved-plans-v2_7/q14.sf100/explain.txt | 4 +- .../q14.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q14/explain.txt | 4 +- .../approved-plans-v2_7/q14/simplified.txt | 2 +- .../q14a.sf100/explain.txt | 4 +- .../q14a.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q14a/explain.txt | 4 +- .../approved-plans-v2_7/q14a/simplified.txt | 2 +- .../approved-plans-v2_7/q35.sf100/explain.txt | 222 ++++---- .../q35.sf100/simplified.txt | 13 +- .../approved-plans-v2_7/q35/explain.txt | 186 ++++--- .../approved-plans-v2_7/q35/simplified.txt | 13 +- .../q35a.sf100/explain.txt | 222 ++++---- .../q35a.sf100/simplified.txt | 31 +- .../approved-plans-v2_7/q35a/explain.txt | 190 ++++--- .../approved-plans-v2_7/q35a/simplified.txt | 31 +- .../approved-plans-v2_7/q5a.sf100/explain.txt | 188 ++++--- .../q5a.sf100/simplified.txt | 9 +- .../approved-plans-v2_7/q5a/explain.txt | 178 +++--- .../approved-plans-v2_7/q5a/simplified.txt | 9 +- .../q70a.sf100/explain.txt | 6 +- .../q70a.sf100/simplified.txt | 4 +- .../approved-plans-v2_7/q70a/explain.txt | 6 +- .../approved-plans-v2_7/q70a/simplified.txt | 4 +- .../sql/DynamicPartitionPruningSuite.scala | 60 +- 106 files changed, 3966 insertions(+), 3554 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/InferDynamicPruningFilters.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index ad567ed9a5d6d..70112ce3e60bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -53,7 +53,6 @@ abstract class Optimizer(catalogManager: CatalogManager) override protected val excludedOnceBatches: Set[String] = Set( "PartitionPruning", - "Infer Filters from PartitionPruning", "Extract Python UDFs") protected def fixedPoint = @@ -922,18 +921,8 @@ object InferFiltersFromGenerate extends Rule[LogicalPlan] { } } -/** - * Generate a list of additional filters from an operator's existing constraint but remove those - * that are either already part of the operator's condition or are part of the operator's child - * constraints. These filters are currently inserted to the existing conditions in the Filter - * operators and on either side of Join operators. - * - * Note: While this optimization is applicable to a lot of types of join, it primarily benefits - * Inner and LeftSemi joins. - */ -object InferFiltersFromConstraints extends Rule[LogicalPlan] - with PredicateHelper with ConstraintHelper { - +abstract class InferFiltersRule extends Rule[LogicalPlan] + with PredicateHelper with ConstraintHelper { def apply(plan: LogicalPlan): LogicalPlan = { if (SQLConf.get.constraintPropagationEnabled) { inferFilters(plan) @@ -942,6 +931,14 @@ object InferFiltersFromConstraints extends Rule[LogicalPlan] } } + protected def getBaseConstraints( + left: LogicalPlan, + right: LogicalPlan, + conditionOpt: Option[Expression]): ExpressionSet = { + left.constraints.union(right.constraints) + .union(ExpressionSet(conditionOpt.map(splitConjunctivePredicates).getOrElse(Nil))) + } + private def inferFilters(plan: LogicalPlan): LogicalPlan = plan transform { case filter @ Filter(condition, child) => val newFilters = filter.constraints -- @@ -952,25 +949,25 @@ object InferFiltersFromConstraints extends Rule[LogicalPlan] filter } - case join @ Join(left, right, joinType, conditionOpt, _) => + case join @ Join(left, right, joinType, _, _) => joinType match { // For inner join, we can infer additional filters for both sides. LeftSemi is kind of an // inner join, it just drops the right side in the final output. case _: InnerLike | LeftSemi => - val allConstraints = getAllConstraints(left, right, conditionOpt) + val allConstraints = getAllConstraints(join) val newLeft = inferNewFilter(left, allConstraints) val newRight = inferNewFilter(right, allConstraints) join.copy(left = newLeft, right = newRight) // For right outer join, we can only infer additional filters for left side. case RightOuter => - val allConstraints = getAllConstraints(left, right, conditionOpt) + val allConstraints = getAllConstraints(join) val newLeft = inferNewFilter(left, allConstraints) join.copy(left = newLeft) // For left join, we can only infer additional filters for right side. case LeftOuter | LeftAnti => - val allConstraints = getAllConstraints(left, right, conditionOpt) + val allConstraints = getAllConstraints(join) val newRight = inferNewFilter(right, allConstraints) join.copy(right = newRight) @@ -978,14 +975,7 @@ object InferFiltersFromConstraints extends Rule[LogicalPlan] } } - private def getAllConstraints( - left: LogicalPlan, - right: LogicalPlan, - conditionOpt: Option[Expression]): ExpressionSet = { - val baseConstraints = left.constraints.union(right.constraints) - .union(ExpressionSet(conditionOpt.map(splitConjunctivePredicates).getOrElse(Nil))) - baseConstraints.union(inferAdditionalConstraints(baseConstraints)) - } + protected def getAllConstraints(join: Join): ExpressionSet private def inferNewFilter(plan: LogicalPlan, constraints: ExpressionSet): LogicalPlan = { val newPredicates = constraints @@ -1001,6 +991,22 @@ object InferFiltersFromConstraints extends Rule[LogicalPlan] } } +/** + * Generate a list of additional filters from an operator's existing constraint but remove those + * that are either already part of the operator's condition or are part of the operator's child + * constraints. These filters are currently inserted to the existing conditions in the Filter + * operators and on either side of Join operators. + * + * Note: While this optimization is applicable to a lot of types of join, it primarily benefits + * Inner and LeftSemi joins. + */ +object InferFiltersFromConstraints extends InferFiltersRule { + override def getAllConstraints(join: Join): ExpressionSet = { + val baseConstraints = getBaseConstraints(join.left, join.right, join.condition) + baseConstraints.union(inferAdditionalConstraints(baseConstraints)) + } +} + /** * Combines all adjacent [[Union]] operators into a single [[Union]]. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala index c4243da7b9e4b..61ee3019d9344 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala @@ -58,10 +58,16 @@ trait ConstraintHelper { * For e.g., if an operator has constraints of the form (`a = 5`, `a = b`), this returns an * additional constraint of the form `b = 5`. */ - def inferAdditionalConstraints(constraints: ExpressionSet): ExpressionSet = { + def inferAdditionalConstraints( + constraints: ExpressionSet, + isInferDynamicPruning: Boolean = false): ExpressionSet = { var inferredConstraints = ExpressionSet() // IsNotNull should be constructed by `constructIsNotNullConstraints`. - val predicates = constraints.filterNot(_.isInstanceOf[IsNotNull]) + val predicates = if (isInferDynamicPruning) { + constraints.filterNot(_.isInstanceOf[IsNotNull]) + } else { + constraints.filterNot(e => e.isInstanceOf[IsNotNull] || e.isInstanceOf[DynamicPruning]) + } predicates.foreach { case eq @ EqualTo(l: Attribute, r: Attribute) => val candidateConstraints = predicates - eq diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index fdaaa587c1fbf..94e4ad786a574 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions import org.apache.spark.sql.execution.datasources.SchemaPruning import org.apache.spark.sql.execution.datasources.v2.{V2ScanRelationPushDown, V2Writes} -import org.apache.spark.sql.execution.dynamicpruning.{CleanupDynamicPruningFilters, PartitionPruning} +import org.apache.spark.sql.execution.dynamicpruning.{CleanupDynamicPruningFilters, InferDynamicPruningFilters, PartitionPruning} import org.apache.spark.sql.execution.python.{ExtractGroupingPythonUDFFromAggregate, ExtractPythonUDFFromAggregate, ExtractPythonUDFs} class SparkOptimizer( @@ -47,12 +47,11 @@ class SparkOptimizer( Batch("Pushdown Filters from PartitionPruning before Inferring Filters", fixedPoint, PushDownPredicates) :+ Batch("Infer Filters from PartitionPruning", Once, - InferFiltersFromConstraints) :+ + InferDynamicPruningFilters) :+ Batch("Pushdown Filters from PartitionPruning after Inferring Filters", fixedPoint, PushDownPredicates) :+ Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, - BooleanSimplification, PruneFilters)) ++ postHocOptimizationBatches :+ Batch("Extract Python UDFs", Once, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/CleanupDynamicPruningFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/CleanupDynamicPruningFilters.scala index c9029b4ad60b5..1cf55a2f2954d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/CleanupDynamicPruningFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/CleanupDynamicPruningFilters.scala @@ -17,13 +17,12 @@ package org.apache.spark.sql.execution.dynamicpruning -import org.apache.spark.sql.catalyst.expressions.{DynamicPruning, DynamicPruningSubquery, Expression, PredicateHelper} +import org.apache.spark.sql.catalyst.expressions.{DynamicPruning, PredicateHelper} import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.execution.dynamicpruning.PartitionPruning._ import org.apache.spark.sql.internal.SQLConf /** @@ -33,36 +32,12 @@ import org.apache.spark.sql.internal.SQLConf */ object CleanupDynamicPruningFilters extends Rule[LogicalPlan] with PredicateHelper { - // Check whether need to remove inferred DPP. - private def isRemoveInferred(condition: Expression, child: LogicalPlan): Boolean = { - splitConjunctivePredicates(condition).exists { - case DynamicPruningSubquery(pruningKey, buildQuery, buildKeys, index, _, _) => - getPartitionTableScan(pruningKey, child).isEmpty || (!SQLConf.get.exchangeReuseEnabled && - !pruningHasBenefit(pruningKey, child, buildKeys(index), buildQuery)) - case _ => false - } - } - override def apply(plan: LogicalPlan): LogicalPlan = { if (!SQLConf.get.dynamicPartitionPruningEnabled) { return plan } plan.transform { - // Remove any DynamicPruning Filters that didn't filter on partition column and - // do not have has benefit. This is inferred by Infer Filters from PartitionPruning. - case f @ Filter(condition, child) - if SQLConf.get.constraintPropagationEnabled && isRemoveInferred(condition, child) => - val newCondition = condition.transform { - case DynamicPruningSubquery(pruningKey, _, _, _, _, _) - if getPartitionTableScan(pruningKey, child).isEmpty => - TrueLiteral - case DynamicPruningSubquery(pruningKey, buildQuery, buildKeys, index, _, _) - if !SQLConf.get.exchangeReuseEnabled && - !pruningHasBenefit(pruningKey, child, buildKeys(index), buildQuery) => - TrueLiteral - } - f.copy(condition = newCondition) // pass through anything that is pushed down into PhysicalOperation case p @ PhysicalOperation(_, _, LogicalRelation(_: HadoopFsRelation, _, _, _)) => p // remove any Filters with DynamicPruning that didn't get pushed down to PhysicalOperation. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/InferDynamicPruningFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/InferDynamicPruningFilters.scala new file mode 100644 index 0000000000000..f2ab3e22dcdf2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/InferDynamicPruningFilters.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.dynamicpruning + +import org.apache.spark.sql.catalyst.expressions.{DynamicPruningSubquery, ExpressionSet} +import org.apache.spark.sql.catalyst.optimizer.InferFiltersRule +import org.apache.spark.sql.catalyst.plans.logical.Join + +object InferDynamicPruningFilters extends InferFiltersRule { + override def getAllConstraints(join: Join): ExpressionSet = { + val baseConstraints = getBaseConstraints(join.left, join.right, join.condition) + val filtered = inferAdditionalConstraints(baseConstraints, true).filter { + case DynamicPruningSubquery( + pruningKey, buildQuery, buildKeys, broadcastKeyIndex, _, _) => + PartitionPruning.getPartitionTableScan(pruningKey, join) match { + case Some(partScan) => + val otherExpr = buildKeys(broadcastKeyIndex) + PartitionPruning.pruningHasBenefit(pruningKey, partScan, otherExpr, buildQuery) + case _ => + false + } + case _ => false + } + baseConstraints.union(filtered) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala index 49f972e681e2b..324003089b5c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala @@ -112,7 +112,7 @@ object PartitionPruning extends Rule[LogicalPlan] with PredicateHelper { * using column statistics if they are available, otherwise we use the config value of * `spark.sql.optimizer.joinFilterRatio`. */ - def pruningHasBenefit( + private[sql] def pruningHasBenefit( partExpr: Expression, partPlan: LogicalPlan, otherExpr: Expression, @@ -148,7 +148,7 @@ object PartitionPruning extends Rule[LogicalPlan] with PredicateHelper { } // the pruning overhead is the total size in bytes of all scan relations - val overhead = otherPlan.collectLeaves().map(_.stats.sizeInBytes).sum + val overhead = otherPlan.collectLeaves().map(_.stats.sizeInBytes).sum.toFloat filterRatio * partPlan.stats.sizeInBytes.toFloat > overhead.toFloat } diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt index c25973f527384..841c7284d9540 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt @@ -1,66 +1,67 @@ == Physical Plan == -TakeOrderedAndProject (55) -+- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * Project (51) - +- * SortMergeJoin Inner (50) - :- * Sort (44) - : +- Exchange (43) - : +- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (35) - : : +- * Filter (34) - : : +- SortMergeJoin ExistenceJoin(exists#1) (33) - : : :- SortMergeJoin ExistenceJoin(exists#2) (25) - : : : :- SortMergeJoin LeftSemi (17) +TakeOrderedAndProject (56) ++- * HashAggregate (55) + +- Exchange (54) + +- * HashAggregate (53) + +- * Project (52) + +- * SortMergeJoin Inner (51) + :- * Sort (45) + : +- Exchange (44) + : +- * Project (43) + : +- * BroadcastHashJoin Inner BuildRight (42) + : :- * Project (36) + : : +- * Filter (35) + : : +- SortMergeJoin ExistenceJoin(exists#1) (34) + : : :- SortMergeJoin ExistenceJoin(exists#2) (26) + : : : :- SortMergeJoin LeftSemi (18) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- * Sort (16) - : : : : +- Exchange (15) - : : : : +- * Project (14) - : : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : : :- * ColumnarToRow (7) - : : : : : +- Scan parquet default.store_sales (6) - : : : : +- BroadcastExchange (12) - : : : : +- * Project (11) - : : : : +- * Filter (10) - : : : : +- * ColumnarToRow (9) - : : : : +- Scan parquet default.date_dim (8) - : : : +- * Sort (24) - : : : +- Exchange (23) - : : : +- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * ColumnarToRow (19) - : : : : +- Scan parquet default.web_sales (18) - : : : +- ReusedExchange (20) - : : +- * Sort (32) - : : +- Exchange (31) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * ColumnarToRow (27) - : : : +- Scan parquet default.catalog_sales (26) - : : +- ReusedExchange (28) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- * Filter (38) - : +- * ColumnarToRow (37) - : +- Scan parquet default.customer_address (36) - +- * Sort (49) - +- Exchange (48) - +- * Filter (47) - +- * ColumnarToRow (46) - +- Scan parquet default.customer_demographics (45) + : : : : +- * Sort (17) + : : : : +- Exchange (16) + : : : : +- * Project (15) + : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * Filter (11) + : : : : +- * ColumnarToRow (10) + : : : : +- Scan parquet default.date_dim (9) + : : : +- * Sort (25) + : : : +- Exchange (24) + : : : +- * Project (23) + : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : :- * ColumnarToRow (20) + : : : : +- Scan parquet default.web_sales (19) + : : : +- ReusedExchange (21) + : : +- * Sort (33) + : : +- Exchange (32) + : : +- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * ColumnarToRow (28) + : : : +- Scan parquet default.catalog_sales (27) + : : +- ReusedExchange (29) + : +- BroadcastExchange (41) + : +- * Project (40) + : +- * Filter (39) + : +- * ColumnarToRow (38) + : +- Scan parquet default.customer_address (37) + +- * Sort (50) + +- Exchange (49) + +- * Filter (48) + +- * ColumnarToRow (47) + +- Scan parquet default.customer_demographics (46) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -68,7 +69,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -83,236 +84,241 @@ Output [2]: [ss_customer_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -(8) Scan parquet default.date_dim +(8) Filter [codegen id : 4] +Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] +Condition : isnotnull(ss_customer_sk#7) + +(9) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(9) ColumnarToRow [codegen id : 3] +(10) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -(10) Filter [codegen id : 3] +(11) Filter [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 2002)) AND (d_moy#12 >= 1)) AND (d_moy#12 <= 4)) AND isnotnull(d_date_sk#10)) -(11) Project [codegen id : 3] +(12) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -(12) BroadcastExchange +(13) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(13) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(14) Project [codegen id : 4] +(15) Project [codegen id : 4] Output [1]: [ss_customer_sk#7] Input [3]: [ss_customer_sk#7, ss_sold_date_sk#8, d_date_sk#10] -(15) Exchange +(16) Exchange Input [1]: [ss_customer_sk#7] Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] -(16) Sort [codegen id : 5] +(17) Sort [codegen id : 5] Input [1]: [ss_customer_sk#7] Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin +(18) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#7] Join condition: None -(18) Scan parquet default.web_sales +(19) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#16), dynamicpruningexpression(ws_sold_date_sk#16 IN dynamicpruning#9)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(20) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] -(20) ReusedExchange [Reuses operator id: 12] +(21) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#10] -(21) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(22) Project [codegen id : 7] +(23) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] -(23) Exchange +(24) Exchange Input [1]: [ws_bill_customer_sk#15] Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] -(24) Sort [codegen id : 8] +(25) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] Arguments: [ws_bill_customer_sk#15 ASC NULLS FIRST], false, 0 -(25) SortMergeJoin +(26) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#15] Join condition: None -(26) Scan parquet default.catalog_sales +(27) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 10] +(28) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] -(28) ReusedExchange [Reuses operator id: 12] +(29) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#10] -(29) BroadcastHashJoin [codegen id : 10] +(30) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#19] Right keys [1]: [d_date_sk#10] Join condition: None -(30) Project [codegen id : 10] +(31) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#18] Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] -(31) Exchange +(32) Exchange Input [1]: [cs_ship_customer_sk#18] Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] -(32) Sort [codegen id : 11] +(33) Sort [codegen id : 11] Input [1]: [cs_ship_customer_sk#18] Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 -(33) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#18] Join condition: None -(34) Filter [codegen id : 13] +(35) Filter [codegen id : 13] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(35) Project [codegen id : 13] +(36) Project [codegen id : 13] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(36) Scan parquet default.customer_address +(37) Scan parquet default.customer_address Output [2]: [ca_address_sk#21, ca_county#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 12] +(38) ColumnarToRow [codegen id : 12] Input [2]: [ca_address_sk#21, ca_county#22] -(38) Filter [codegen id : 12] +(39) Filter [codegen id : 12] Input [2]: [ca_address_sk#21, ca_county#22] Condition : (ca_county#22 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#21)) -(39) Project [codegen id : 12] +(40) Project [codegen id : 12] Output [1]: [ca_address_sk#21] Input [2]: [ca_address_sk#21, ca_county#22] -(40) BroadcastExchange +(41) BroadcastExchange Input [1]: [ca_address_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] -(41) BroadcastHashJoin [codegen id : 13] +(42) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#21] Join condition: None -(42) Project [codegen id : 13] +(43) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#21] -(43) Exchange +(44) Exchange Input [1]: [c_current_cdemo_sk#4] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#24] -(44) Sort [codegen id : 14] +(45) Sort [codegen id : 14] Input [1]: [c_current_cdemo_sk#4] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(45) Scan parquet default.customer_demographics +(46) Scan parquet default.customer_demographics Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 15] +(47) ColumnarToRow [codegen id : 15] Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -(47) Filter [codegen id : 15] +(48) Filter [codegen id : 15] Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Condition : isnotnull(cd_demo_sk#25) -(48) Exchange +(49) Exchange Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#34] -(49) Sort [codegen id : 16] +(50) Sort [codegen id : 16] Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 -(50) SortMergeJoin [codegen id : 17] +(51) SortMergeJoin [codegen id : 17] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#25] Join condition: None -(51) Project [codegen id : 17] +(52) Project [codegen id : 17] Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -(52) HashAggregate [codegen id : 17] +(53) HashAggregate [codegen id : 17] Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#35] Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -(53) Exchange +(54) Exchange Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] -(54) HashAggregate [codegen id : 18] +(55) HashAggregate [codegen id : 18] Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#38] Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] -(55) TakeOrderedAndProject +(56) TakeOrderedAndProject Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (56) +ReusedExchange (57) -(56) ReusedExchange [Reuses operator id: 12] +(57) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt index c10844dccad8f..fa8f3c241be38 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt @@ -26,7 +26,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -37,11 +37,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #5 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 07bcd309e9ce6..0e011f1c44c0b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -1,58 +1,59 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (30) - : : +- * Filter (29) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (28) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (21) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (14) +TakeOrderedAndProject (48) ++- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (31) + : : +- * Filter (30) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (29) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (22) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (13) - : : : : +- * Project (12) - : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (10) - : : : : +- * Project (9) - : : : : +- * Filter (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.date_dim (6) - : : : +- BroadcastExchange (20) - : : : +- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * ColumnarToRow (16) - : : : : +- Scan parquet default.web_sales (15) - : : : +- ReusedExchange (17) - : : +- BroadcastExchange (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (24) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.customer_address (31) - +- BroadcastExchange (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.customer_demographics (38) + : : : : +- BroadcastExchange (14) + : : : : +- * Project (13) + : : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.store_sales (4) + : : : : +- BroadcastExchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.date_dim (7) + : : : +- BroadcastExchange (21) + : : : +- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * ColumnarToRow (17) + : : : : +- Scan parquet default.web_sales (16) + : : : +- ReusedExchange (18) + : : +- BroadcastExchange (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * ColumnarToRow (24) + : : : +- Scan parquet default.catalog_sales (23) + : : +- ReusedExchange (25) + : +- BroadcastExchange (36) + : +- * Project (35) + : +- * Filter (34) + : +- * ColumnarToRow (33) + : +- Scan parquet default.customer_address (32) + +- BroadcastExchange (42) + +- * Filter (41) + +- * ColumnarToRow (40) + +- Scan parquet default.customer_demographics (39) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -60,219 +61,224 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) Scan parquet default.date_dim +(6) Filter [codegen id : 2] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#6) + +(7) Scan parquet default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 1] +(8) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(8) Filter [codegen id : 1] +(9) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) -(9) Project [codegen id : 1] +(10) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(10) BroadcastExchange +(11) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] -(11) BroadcastHashJoin [codegen id : 2] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join condition: None -(12) Project [codegen id : 2] +(13) Project [codegen id : 2] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(13) BroadcastExchange +(14) BroadcastExchange Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(14) BroadcastHashJoin [codegen id : 9] +(15) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join condition: None -(15) Scan parquet default.web_sales +(16) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#15), dynamicpruningexpression(ws_sold_date_sk#15 IN dynamicpruning#8)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 4] +(17) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] -(17) ReusedExchange [Reuses operator id: 10] +(18) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#9] -(18) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] Right keys [1]: [d_date_sk#9] Join condition: None -(19) Project [codegen id : 4] +(20) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] -(20) BroadcastExchange +(21) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(21) BroadcastHashJoin [codegen id : 9] +(22) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#14] Join condition: None -(22) Scan parquet default.catalog_sales +(23) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 6] +(24) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] -(24) ReusedExchange [Reuses operator id: 10] +(25) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#9] -(25) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#9] Join condition: None -(26) Project [codegen id : 6] +(27) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#17] Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] -(27) BroadcastExchange +(28) BroadcastExchange Input [1]: [cs_ship_customer_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(28) BroadcastHashJoin [codegen id : 9] +(29) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#17] Join condition: None -(29) Filter [codegen id : 9] +(30) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(30) Project [codegen id : 9] +(31) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(31) Scan parquet default.customer_address +(32) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 7] +(33) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#20, ca_county#21] -(33) Filter [codegen id : 7] +(34) Filter [codegen id : 7] Input [2]: [ca_address_sk#20, ca_county#21] Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) -(34) Project [codegen id : 7] +(35) Project [codegen id : 7] Output [1]: [ca_address_sk#20] Input [2]: [ca_address_sk#20, ca_county#21] -(35) BroadcastExchange +(36) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(36) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join condition: None -(37) Project [codegen id : 9] +(38) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] -(38) Scan parquet default.customer_demographics +(39) Scan parquet default.customer_demographics Output [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 8] +(40) ColumnarToRow [codegen id : 8] Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(40) Filter [codegen id : 8] +(41) Filter [codegen id : 8] Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Condition : isnotnull(cd_demo_sk#23) -(41) BroadcastExchange +(42) BroadcastExchange Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(42) BroadcastHashJoin [codegen id : 9] +(43) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join condition: None -(43) Project [codegen id : 9] +(44) Project [codegen id : 9] Output [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(44) HashAggregate [codegen id : 9] +(45) HashAggregate [codegen id : 9] Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#33] Results [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -(45) Exchange +(46) Exchange Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#35] -(46) HashAggregate [codegen id : 10] +(47) HashAggregate [codegen id : 10] Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#36] Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#36 AS cnt1#37, cd_purchase_estimate#27, count(1)#36 AS cnt2#38, cd_credit_rating#28, count(1)#36 AS cnt3#39, cd_dep_count#29, count(1)#36 AS cnt4#40, cd_dep_employed_count#30, count(1)#36 AS cnt5#41, cd_dep_college_count#31, count(1)#36 AS cnt6#42] -(47) TakeOrderedAndProject +(48) TakeOrderedAndProject Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -ReusedExchange (48) +ReusedExchange (49) -(48) ReusedExchange [Reuses operator id: 10] +(49) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#9] -Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 261bb6152b22a..78861b51e4f6d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -23,11 +23,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) 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 d4ab3f60494d9..05b0c9b7dfff5 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 @@ -162,7 +162,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 20] @@ -170,7 +170,7 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +Condition : (((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_item_sk#7)) (9) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] 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 4ee1ff31e9beb..341784cbfdeae 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 @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (20) 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] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 85a3a63413f92..8208efcb1755a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -135,7 +135,7 @@ Condition : isnotnull(ss_item_sk#1) 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)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -143,7 +143,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] 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)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (7) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index f9b76560e363c..a66dd29c4927d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] 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 76f5ac48f96cc..3109147596cf8 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 @@ -144,7 +144,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 20] @@ -152,7 +152,7 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +Condition : (((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_item_sk#7)) (9) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] 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 1d5907e319658..be55d3b2fecfc 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 @@ -75,7 +75,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (20) 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] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 01bcb83622844..7315c864712eb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -120,7 +120,7 @@ Condition : isnotnull(ss_item_sk#1) 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)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -128,7 +128,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] 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)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (7) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 3507d76bcf9b9..592d91ddf08fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -67,7 +67,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index adfa3e44f996d..154c74df64b64 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -1,57 +1,59 @@ == Physical Plan == -* Sort (46) -+- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- SortMergeJoin LeftSemi (12) +* Sort (48) ++- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- SortMergeJoin LeftAnti (21) + : : : :- * Project (14) + : : : : +- SortMergeJoin LeftSemi (13) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.catalog_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * Project (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.catalog_sales (7) - : : : +- * Sort (18) - : : : +- Exchange (17) - : : : +- * Project (16) - : : : +- * ColumnarToRow (15) - : : : +- Scan parquet default.catalog_returns (14) - : : +- BroadcastExchange (24) - : : +- * Project (23) - : : +- * Filter (22) - : : +- * ColumnarToRow (21) - : : +- Scan parquet default.customer_address (20) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet default.call_center (27) - +- BroadcastExchange (38) - +- * Project (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.date_dim (34) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.catalog_sales (7) + : : : +- * Sort (20) + : : : +- Exchange (19) + : : : +- * Project (18) + : : : +- * Filter (17) + : : : +- * ColumnarToRow (16) + : : : +- Scan parquet default.catalog_returns (15) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet default.customer_address (22) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet default.call_center (29) + +- BroadcastExchange (40) + +- * Project (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet default.date_dim (36) (1) Scan parquet default.catalog_sales Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] +PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -59,7 +61,7 @@ Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho (3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) +Condition : ((((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_order_number#5)) AND isnotnull(cs_warehouse_sk#4)) (4) Project [codegen id : 1] Output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -77,184 +79,194 @@ Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -(9) Project [codegen id : 3] +(9) Filter [codegen id : 3] +Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Condition : (isnotnull(cs_order_number#5) AND isnotnull(cs_warehouse_sk#4)) + +(10) Project [codegen id : 3] Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -(10) Exchange +(11) Exchange Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] -(11) Sort [codegen id : 4] +(12) Sort [codegen id : 4] Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin +(13) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#5#11] Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) -(13) Project [codegen id : 5] +(14) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(14) Scan parquet default.catalog_returns +(15) Scan parquet default.catalog_returns Output [2]: [cr_order_number#13, cr_returned_date_sk#14] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_order_number)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 6] +(16) ColumnarToRow [codegen id : 6] +Input [2]: [cr_order_number#13, cr_returned_date_sk#14] + +(17) Filter [codegen id : 6] Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Condition : isnotnull(cr_order_number#13) -(16) Project [codegen id : 6] +(18) Project [codegen id : 6] Output [1]: [cr_order_number#13] Input [2]: [cr_order_number#13, cr_returned_date_sk#14] -(17) Exchange +(19) Exchange Input [1]: [cr_order_number#13] Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] -(18) Sort [codegen id : 7] +(20) Sort [codegen id : 7] Input [1]: [cr_order_number#13] Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin +(21) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#13] Join condition: None -(20) Scan parquet default.customer_address +(22) Scan parquet default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(21) ColumnarToRow [codegen id : 8] +(23) ColumnarToRow [codegen id : 8] Input [2]: [ca_address_sk#16, ca_state#17] -(22) Filter [codegen id : 8] +(24) Filter [codegen id : 8] Input [2]: [ca_address_sk#16, ca_state#17] Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_address_sk#16)) -(23) Project [codegen id : 8] +(25) Project [codegen id : 8] Output [1]: [ca_address_sk#16] Input [2]: [ca_address_sk#16, ca_state#17] -(24) BroadcastExchange +(26) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(25) BroadcastHashJoin [codegen id : 11] +(27) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join condition: None -(26) Project [codegen id : 11] +(28) Project [codegen id : 11] Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -(27) Scan parquet default.call_center +(29) Scan parquet default.call_center Output [2]: [cc_call_center_sk#19, cc_county#20] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 9] Input [2]: [cc_call_center_sk#19, cc_county#20] -(29) Filter [codegen id : 9] +(31) Filter [codegen id : 9] Input [2]: [cc_call_center_sk#19, cc_county#20] Condition : ((isnotnull(cc_county#20) AND (cc_county#20 = Williamson County)) AND isnotnull(cc_call_center_sk#19)) -(30) Project [codegen id : 9] +(32) Project [codegen id : 9] Output [1]: [cc_call_center_sk#19] Input [2]: [cc_call_center_sk#19, cc_county#20] -(31) BroadcastExchange +(33) BroadcastExchange Input [1]: [cc_call_center_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(32) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#19] Join condition: None -(33) Project [codegen id : 11] +(35) Project [codegen id : 11] Output [4]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#19] -(34) Scan parquet default.date_dim +(36) Scan parquet default.date_dim Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 10] Input [2]: [d_date_sk#22, d_date#23] -(36) Filter [codegen id : 10] +(38) Filter [codegen id : 10] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11719)) AND (d_date#23 <= 11779)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2002-02-01)) AND (d_date#23 <= 2002-04-02)) AND isnotnull(d_date_sk#22)) -(37) Project [codegen id : 10] +(39) Project [codegen id : 10] Output [1]: [d_date_sk#22] Input [2]: [d_date_sk#22, d_date#23] -(38) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(39) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#22] Join condition: None -(40) Project [codegen id : 11] +(42) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#22] -(41) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] Results [3]: [cs_order_number#5, sum#27, sum#28] -(42) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#27, sum#28] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] Results [3]: [cs_order_number#5, sum#27, sum#28] -(43) HashAggregate [codegen id : 11] +(45) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#27, sum#28] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] Results [3]: [sum#27, sum#28, count#30] -(44) Exchange +(46) Exchange Input [3]: [sum#27, sum#28, count#30] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] -(45) HashAggregate [codegen id : 12] +(47) HashAggregate [codegen id : 12] Input [3]: [sum#27, sum#28, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] -(46) Sort [codegen id : 12] +(48) Sort [codegen id : 12] Input [3]: [order count #32, total shipping cost #33, total net profit #34] Arguments: [order count #32 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt index 393a5baec4400..5627d2f61e900 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (12) Exchange [cs_order_number] #2 WholeStageCodegen (1) Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_warehouse_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] @@ -35,18 +35,20 @@ WholeStageCodegen (12) Exchange [cs_order_number] #3 WholeStageCodegen (3) Project [cs_warehouse_sk,cs_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + Filter [cs_order_number,cs_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] WholeStageCodegen (7) Sort [cr_order_number] InputAdapter Exchange [cr_order_number] #4 WholeStageCodegen (6) Project [cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] + Filter [cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 5640564564396..392a265dcff07 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -1,57 +1,59 @@ == Physical Plan == -* Sort (46) -+- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- SortMergeJoin LeftSemi (12) +* Sort (48) ++- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- SortMergeJoin LeftAnti (21) + : : : :- * Project (14) + : : : : +- SortMergeJoin LeftSemi (13) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.catalog_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * Project (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.catalog_sales (7) - : : : +- * Sort (18) - : : : +- Exchange (17) - : : : +- * Project (16) - : : : +- * ColumnarToRow (15) - : : : +- Scan parquet default.catalog_returns (14) - : : +- BroadcastExchange (24) - : : +- * Project (23) - : : +- * Filter (22) - : : +- * ColumnarToRow (21) - : : +- Scan parquet default.date_dim (20) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet default.customer_address (27) - +- BroadcastExchange (38) - +- * Project (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.call_center (34) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.catalog_sales (7) + : : : +- * Sort (20) + : : : +- Exchange (19) + : : : +- * Project (18) + : : : +- * Filter (17) + : : : +- * ColumnarToRow (16) + : : : +- Scan parquet default.catalog_returns (15) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet default.date_dim (22) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet default.customer_address (29) + +- BroadcastExchange (40) + +- * Project (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet default.call_center (36) (1) Scan parquet default.catalog_sales Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] +PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -59,7 +61,7 @@ Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho (3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) +Condition : ((((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_order_number#5)) AND isnotnull(cs_warehouse_sk#4)) (4) Project [codegen id : 1] Output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -77,184 +79,194 @@ Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -(9) Project [codegen id : 3] +(9) Filter [codegen id : 3] +Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Condition : (isnotnull(cs_order_number#5) AND isnotnull(cs_warehouse_sk#4)) + +(10) Project [codegen id : 3] Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -(10) Exchange +(11) Exchange Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] -(11) Sort [codegen id : 4] +(12) Sort [codegen id : 4] Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin +(13) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#5#11] Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) -(13) Project [codegen id : 5] +(14) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(14) Scan parquet default.catalog_returns +(15) Scan parquet default.catalog_returns Output [2]: [cr_order_number#13, cr_returned_date_sk#14] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_order_number)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 6] +(16) ColumnarToRow [codegen id : 6] +Input [2]: [cr_order_number#13, cr_returned_date_sk#14] + +(17) Filter [codegen id : 6] Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Condition : isnotnull(cr_order_number#13) -(16) Project [codegen id : 6] +(18) Project [codegen id : 6] Output [1]: [cr_order_number#13] Input [2]: [cr_order_number#13, cr_returned_date_sk#14] -(17) Exchange +(19) Exchange Input [1]: [cr_order_number#13] Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] -(18) Sort [codegen id : 7] +(20) Sort [codegen id : 7] Input [1]: [cr_order_number#13] Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin +(21) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#13] Join condition: None -(20) Scan parquet default.date_dim +(22) Scan parquet default.date_dim Output [2]: [d_date_sk#16, d_date#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(21) ColumnarToRow [codegen id : 8] +(23) ColumnarToRow [codegen id : 8] Input [2]: [d_date_sk#16, d_date#17] -(22) Filter [codegen id : 8] +(24) Filter [codegen id : 8] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 11719)) AND (d_date#17 <= 11779)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2002-02-01)) AND (d_date#17 <= 2002-04-02)) AND isnotnull(d_date_sk#16)) -(23) Project [codegen id : 8] +(25) Project [codegen id : 8] Output [1]: [d_date_sk#16] Input [2]: [d_date_sk#16, d_date#17] -(24) BroadcastExchange +(26) BroadcastExchange Input [1]: [d_date_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(25) BroadcastHashJoin [codegen id : 11] +(27) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#16] Join condition: None -(26) Project [codegen id : 11] +(28) Project [codegen id : 11] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#16] -(27) Scan parquet default.customer_address +(29) Scan parquet default.customer_address Output [2]: [ca_address_sk#19, ca_state#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 9] Input [2]: [ca_address_sk#19, ca_state#20] -(29) Filter [codegen id : 9] +(31) Filter [codegen id : 9] Input [2]: [ca_address_sk#19, ca_state#20] Condition : ((isnotnull(ca_state#20) AND (ca_state#20 = GA)) AND isnotnull(ca_address_sk#19)) -(30) Project [codegen id : 9] +(32) Project [codegen id : 9] Output [1]: [ca_address_sk#19] Input [2]: [ca_address_sk#19, ca_state#20] -(31) BroadcastExchange +(33) BroadcastExchange Input [1]: [ca_address_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(32) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#19] Join condition: None -(33) Project [codegen id : 11] +(35) Project [codegen id : 11] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#19] -(34) Scan parquet default.call_center +(36) Scan parquet default.call_center Output [2]: [cc_call_center_sk#22, cc_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 10] Input [2]: [cc_call_center_sk#22, cc_county#23] -(36) Filter [codegen id : 10] +(38) Filter [codegen id : 10] Input [2]: [cc_call_center_sk#22, cc_county#23] Condition : ((isnotnull(cc_county#23) AND (cc_county#23 = Williamson County)) AND isnotnull(cc_call_center_sk#22)) -(37) Project [codegen id : 10] +(39) Project [codegen id : 10] Output [1]: [cc_call_center_sk#22] Input [2]: [cc_call_center_sk#22, cc_county#23] -(38) BroadcastExchange +(40) BroadcastExchange Input [1]: [cc_call_center_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(39) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#22] Join condition: None -(40) Project [codegen id : 11] +(42) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#22] -(41) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] Results [3]: [cs_order_number#5, sum#27, sum#28] -(42) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#27, sum#28] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] Results [3]: [cs_order_number#5, sum#27, sum#28] -(43) HashAggregate [codegen id : 11] +(45) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#27, sum#28] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] Results [3]: [sum#27, sum#28, count#30] -(44) Exchange +(46) Exchange Input [3]: [sum#27, sum#28, count#30] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] -(45) HashAggregate [codegen id : 12] +(47) HashAggregate [codegen id : 12] Input [3]: [sum#27, sum#28, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] -(46) Sort [codegen id : 12] +(48) Sort [codegen id : 12] Input [3]: [order count #32, total shipping cost #33, total net profit #34] Arguments: [order count #32 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index 5edd1426dd8f1..90655232d690e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (12) Exchange [cs_order_number] #2 WholeStageCodegen (1) Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_warehouse_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] @@ -35,18 +35,20 @@ WholeStageCodegen (12) Exchange [cs_order_number] #3 WholeStageCodegen (3) Project [cs_warehouse_sk,cs_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + Filter [cs_order_number,cs_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] WholeStageCodegen (7) Sort [cr_order_number] InputAdapter Exchange [cr_order_number] #4 WholeStageCodegen (6) Project [cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] + Filter [cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt index aa96edd4c7fba..46acb1a9eebdb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt @@ -1,102 +1,104 @@ == Physical Plan == -* HashAggregate (98) -+- Exchange (97) - +- * HashAggregate (96) - +- Union (95) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (52) - : : +- SortMergeJoin LeftSemi (51) - : : :- * Sort (32) - : : : +- Exchange (31) - : : : +- * Project (30) - : : : +- SortMergeJoin LeftSemi (29) - : : : :- * Sort (4) - : : : : +- Exchange (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.catalog_sales (1) - : : : +- * Sort (28) - : : : +- * Project (27) - : : : +- * Filter (26) - : : : +- * HashAggregate (25) - : : : +- * HashAggregate (24) - : : : +- * Project (23) - : : : +- * SortMergeJoin Inner (22) - : : : :- * Sort (16) - : : : : +- Exchange (15) - : : : : +- * Project (14) - : : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : : :- * Filter (7) - : : : : : +- * ColumnarToRow (6) - : : : : : +- Scan parquet default.store_sales (5) - : : : : +- BroadcastExchange (12) - : : : : +- * Project (11) - : : : : +- * Filter (10) - : : : : +- * ColumnarToRow (9) - : : : : +- Scan parquet default.date_dim (8) - : : : +- * Sort (21) - : : : +- Exchange (20) - : : : +- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.item (17) - : : +- * Sort (50) - : : +- * Project (49) - : : +- * Filter (48) - : : +- * HashAggregate (47) - : : +- * HashAggregate (46) - : : +- * Project (45) - : : +- * SortMergeJoin Inner (44) - : : :- * Sort (38) - : : : +- Exchange (37) - : : : +- * Project (36) - : : : +- * Filter (35) - : : : +- * ColumnarToRow (34) - : : : +- Scan parquet default.store_sales (33) - : : +- * Sort (43) - : : +- Exchange (42) - : : +- * Filter (41) - : : +- * ColumnarToRow (40) - : : +- Scan parquet default.customer (39) - : +- BroadcastExchange (57) - : +- * Project (56) - : +- * Filter (55) - : +- * ColumnarToRow (54) - : +- Scan parquet default.date_dim (53) - +- * Project (94) - +- * BroadcastHashJoin Inner BuildRight (93) - :- * Project (91) - : +- SortMergeJoin LeftSemi (90) - : :- * Sort (78) - : : +- Exchange (77) - : : +- * Project (76) - : : +- SortMergeJoin LeftSemi (75) - : : :- * Sort (63) - : : : +- Exchange (62) - : : : +- * ColumnarToRow (61) - : : : +- Scan parquet default.web_sales (60) - : : +- * Sort (74) - : : +- * Project (73) - : : +- * Filter (72) - : : +- * HashAggregate (71) - : : +- * HashAggregate (70) - : : +- * Project (69) - : : +- * SortMergeJoin Inner (68) - : : :- * Sort (65) - : : : +- ReusedExchange (64) - : : +- * Sort (67) - : : +- ReusedExchange (66) - : +- * Sort (89) - : +- * Project (88) - : +- * Filter (87) - : +- * HashAggregate (86) - : +- * HashAggregate (85) - : +- * Project (84) - : +- * SortMergeJoin Inner (83) - : :- * Sort (80) - : : +- ReusedExchange (79) - : +- * Sort (82) - : +- ReusedExchange (81) - +- ReusedExchange (92) +* HashAggregate (100) ++- Exchange (99) + +- * HashAggregate (98) + +- Union (97) + :- * Project (60) + : +- * BroadcastHashJoin Inner BuildRight (59) + : :- * Project (53) + : : +- SortMergeJoin LeftSemi (52) + : : :- * Sort (33) + : : : +- Exchange (32) + : : : +- * Project (31) + : : : +- SortMergeJoin LeftSemi (30) + : : : :- * Sort (5) + : : : : +- Exchange (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.catalog_sales (1) + : : : +- * Sort (29) + : : : +- * Project (28) + : : : +- * Filter (27) + : : : +- * HashAggregate (26) + : : : +- * HashAggregate (25) + : : : +- * Project (24) + : : : +- * SortMergeJoin Inner (23) + : : : :- * Sort (17) + : : : : +- Exchange (16) + : : : : +- * Project (15) + : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * Filter (11) + : : : : +- * ColumnarToRow (10) + : : : : +- Scan parquet default.date_dim (9) + : : : +- * Sort (22) + : : : +- Exchange (21) + : : : +- * Filter (20) + : : : +- * ColumnarToRow (19) + : : : +- Scan parquet default.item (18) + : : +- * Sort (51) + : : +- * Project (50) + : : +- * Filter (49) + : : +- * HashAggregate (48) + : : +- * HashAggregate (47) + : : +- * Project (46) + : : +- * SortMergeJoin Inner (45) + : : :- * Sort (39) + : : : +- Exchange (38) + : : : +- * Project (37) + : : : +- * Filter (36) + : : : +- * ColumnarToRow (35) + : : : +- Scan parquet default.store_sales (34) + : : +- * Sort (44) + : : +- Exchange (43) + : : +- * Filter (42) + : : +- * ColumnarToRow (41) + : : +- Scan parquet default.customer (40) + : +- BroadcastExchange (58) + : +- * Project (57) + : +- * Filter (56) + : +- * ColumnarToRow (55) + : +- Scan parquet default.date_dim (54) + +- * Project (96) + +- * BroadcastHashJoin Inner BuildRight (95) + :- * Project (93) + : +- SortMergeJoin LeftSemi (92) + : :- * Sort (80) + : : +- Exchange (79) + : : +- * Project (78) + : : +- SortMergeJoin LeftSemi (77) + : : :- * Sort (65) + : : : +- Exchange (64) + : : : +- * Filter (63) + : : : +- * ColumnarToRow (62) + : : : +- Scan parquet default.web_sales (61) + : : +- * Sort (76) + : : +- * Project (75) + : : +- * Filter (74) + : : +- * HashAggregate (73) + : : +- * HashAggregate (72) + : : +- * Project (71) + : : +- * SortMergeJoin Inner (70) + : : :- * Sort (67) + : : : +- ReusedExchange (66) + : : +- * Sort (69) + : : +- ReusedExchange (68) + : +- * Sort (91) + : +- * Project (90) + : +- * Filter (89) + : +- * HashAggregate (88) + : +- * HashAggregate (87) + : +- * Project (86) + : +- * SortMergeJoin Inner (85) + : :- * Sort (82) + : : +- ReusedExchange (81) + : +- * Sort (84) + : +- ReusedExchange (83) + +- ReusedExchange (94) (1) Scan parquet default.catalog_sales @@ -104,20 +106,25 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(3) Exchange +(3) Filter [codegen id : 1] +Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) + +(4) Exchange Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: hashpartitioning(cs_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#7] -(4) Sort [codegen id : 2] +(5) Sort [codegen id : 2] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 -(5) Scan parquet default.store_sales +(6) Scan parquet default.store_sales Output [2]: [ss_item_sk#8, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -125,417 +132,422 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) ColumnarToRow [codegen id : 4] +(7) ColumnarToRow [codegen id : 4] Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] -(7) Filter [codegen id : 4] +(8) Filter [codegen id : 4] Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] Condition : isnotnull(ss_item_sk#8) -(8) Scan parquet default.date_dim +(9) Scan parquet default.date_dim Output [3]: [d_date_sk#11, d_date#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(9) ColumnarToRow [codegen id : 3] +(10) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#11, d_date#12, d_year#13] -(10) Filter [codegen id : 3] +(11) Filter [codegen id : 3] Input [3]: [d_date_sk#11, d_date#12, d_year#13] Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) -(11) Project [codegen id : 3] +(12) Project [codegen id : 3] Output [2]: [d_date_sk#11, d_date#12] Input [3]: [d_date_sk#11, d_date#12, d_year#13] -(12) BroadcastExchange +(13) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(13) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join condition: None -(14) Project [codegen id : 4] +(15) Project [codegen id : 4] Output [2]: [ss_item_sk#8, d_date#12] Input [4]: [ss_item_sk#8, ss_sold_date_sk#9, d_date_sk#11, d_date#12] -(15) Exchange +(16) Exchange Input [2]: [ss_item_sk#8, d_date#12] Arguments: hashpartitioning(ss_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#15] -(16) Sort [codegen id : 5] +(17) Sort [codegen id : 5] Input [2]: [ss_item_sk#8, d_date#12] Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 -(17) Scan parquet default.item +(18) Scan parquet default.item Output [2]: [i_item_sk#16, i_item_desc#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 6] Input [2]: [i_item_sk#16, i_item_desc#17] -(19) Filter [codegen id : 6] +(20) Filter [codegen id : 6] Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(20) Exchange +(21) Exchange Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: hashpartitioning(i_item_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] -(21) Sort [codegen id : 7] +(22) Sort [codegen id : 7] Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin [codegen id : 8] +(23) SortMergeJoin [codegen id : 8] Left keys [1]: [ss_item_sk#8] Right keys [1]: [i_item_sk#16] Join condition: None -(23) Project [codegen id : 8] +(24) Project [codegen id : 8] Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] -(24) HashAggregate [codegen id : 8] +(25) HashAggregate [codegen id : 8] Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#19, i_item_sk#16, d_date#12] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#20] Results [4]: [substr(i_item_desc#17, 1, 30)#19, i_item_sk#16, d_date#12, count#21] -(25) HashAggregate [codegen id : 8] +(26) HashAggregate [codegen id : 8] Input [4]: [substr(i_item_desc#17, 1, 30)#19, i_item_sk#16, d_date#12, count#21] Keys [3]: [substr(i_item_desc#17, 1, 30)#19, i_item_sk#16, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#22] Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#22 AS count(1)#24] -(26) Filter [codegen id : 8] +(27) Filter [codegen id : 8] Input [2]: [item_sk#23, count(1)#24] Condition : (count(1)#24 > 4) -(27) Project [codegen id : 8] +(28) Project [codegen id : 8] Output [1]: [item_sk#23] Input [2]: [item_sk#23, count(1)#24] -(28) Sort [codegen id : 8] +(29) Sort [codegen id : 8] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin +(30) SortMergeJoin Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#23] Join condition: None -(30) Project [codegen id : 9] +(31) Project [codegen id : 9] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(31) Exchange +(32) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#25] -(32) Sort [codegen id : 10] +(33) Sort [codegen id : 10] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(33) Scan parquet default.store_sales +(34) Scan parquet default.store_sales Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 11] +(35) ColumnarToRow [codegen id : 11] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -(35) Filter [codegen id : 11] +(36) Filter [codegen id : 11] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Condition : isnotnull(ss_customer_sk#26) -(36) Project [codegen id : 11] +(37) Project [codegen id : 11] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -(37) Exchange +(38) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#29] -(38) Sort [codegen id : 12] +(39) Sort [codegen id : 12] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 -(39) Scan parquet default.customer +(40) Scan parquet default.customer Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(40) ColumnarToRow [codegen id : 13] +(41) ColumnarToRow [codegen id : 13] Input [1]: [c_customer_sk#30] -(41) Filter [codegen id : 13] +(42) Filter [codegen id : 13] Input [1]: [c_customer_sk#30] Condition : isnotnull(c_customer_sk#30) -(42) Exchange +(43) Exchange Input [1]: [c_customer_sk#30] Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#31] -(43) Sort [codegen id : 14] +(44) Sort [codegen id : 14] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(44) SortMergeJoin [codegen id : 15] +(45) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#30] Join condition: None -(45) Project [codegen id : 15] +(46) Project [codegen id : 15] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -(46) HashAggregate [codegen id : 15] +(47) HashAggregate [codegen id : 15] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#32, isEmpty#33] Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] -(47) HashAggregate [codegen id : 15] +(48) HashAggregate [codegen id : 15] Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -(48) Filter [codegen id : 15] +(49) Filter [codegen id : 15] Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) -(49) Project [codegen id : 15] +(50) Project [codegen id : 15] Output [1]: [c_customer_sk#30] Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -(50) Sort [codegen id : 15] +(51) Sort [codegen id : 15] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(51) SortMergeJoin +(52) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#30] Join condition: None -(52) Project [codegen id : 17] +(53) Project [codegen id : 17] Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(53) Scan parquet default.date_dim +(54) Scan parquet default.date_dim Output [3]: [d_date_sk#11, d_year#13, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 16] +(55) ColumnarToRow [codegen id : 16] Input [3]: [d_date_sk#11, d_year#13, d_moy#40] -(55) Filter [codegen id : 16] +(56) Filter [codegen id : 16] Input [3]: [d_date_sk#11, d_year#13, d_moy#40] Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#40)) AND (d_year#13 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#11)) -(56) Project [codegen id : 16] +(57) Project [codegen id : 16] Output [1]: [d_date_sk#11] Input [3]: [d_date_sk#11, d_year#13, d_moy#40] -(57) BroadcastExchange +(58) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] -(58) BroadcastHashJoin [codegen id : 17] +(59) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#11] Join condition: None -(59) Project [codegen id : 17] +(60) Project [codegen id : 17] Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#11] -(60) Scan parquet default.web_sales +(61) Scan parquet default.web_sales Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 18] +(62) ColumnarToRow [codegen id : 18] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] + +(63) Filter [codegen id : 18] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Condition : (isnotnull(ws_item_sk#43) AND isnotnull(ws_bill_customer_sk#44)) -(62) Exchange +(64) Exchange Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: hashpartitioning(ws_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#48] -(63) Sort [codegen id : 19] +(65) Sort [codegen id : 19] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: [ws_item_sk#43 ASC NULLS FIRST], false, 0 -(64) ReusedExchange [Reuses operator id: unknown] +(66) ReusedExchange [Reuses operator id: unknown] Output [2]: [ss_item_sk#8, d_date#12] -(65) Sort [codegen id : 22] +(67) Sort [codegen id : 22] Input [2]: [ss_item_sk#8, d_date#12] Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 -(66) ReusedExchange [Reuses operator id: 20] +(68) ReusedExchange [Reuses operator id: 21] Output [2]: [i_item_sk#16, i_item_desc#17] -(67) Sort [codegen id : 24] +(69) Sort [codegen id : 24] Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 -(68) SortMergeJoin [codegen id : 25] +(70) SortMergeJoin [codegen id : 25] Left keys [1]: [ss_item_sk#8] Right keys [1]: [i_item_sk#16] Join condition: None -(69) Project [codegen id : 25] +(71) Project [codegen id : 25] Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] -(70) HashAggregate [codegen id : 25] +(72) HashAggregate [codegen id : 25] Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#50] Results [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] -(71) HashAggregate [codegen id : 25] +(73) HashAggregate [codegen id : 25] Input [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] Keys [3]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#52] Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#52 AS count(1)#53] -(72) Filter [codegen id : 25] +(74) Filter [codegen id : 25] Input [2]: [item_sk#23, count(1)#53] Condition : (count(1)#53 > 4) -(73) Project [codegen id : 25] +(75) Project [codegen id : 25] Output [1]: [item_sk#23] Input [2]: [item_sk#23, count(1)#53] -(74) Sort [codegen id : 25] +(76) Sort [codegen id : 25] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 -(75) SortMergeJoin +(77) SortMergeJoin Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#23] Join condition: None -(76) Project [codegen id : 26] +(78) Project [codegen id : 26] Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(77) Exchange +(79) Exchange Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#54] -(78) Sort [codegen id : 27] +(80) Sort [codegen id : 27] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 -(79) ReusedExchange [Reuses operator id: 37] +(81) ReusedExchange [Reuses operator id: 38] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -(80) Sort [codegen id : 29] +(82) Sort [codegen id : 29] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 -(81) ReusedExchange [Reuses operator id: 42] +(83) ReusedExchange [Reuses operator id: 43] Output [1]: [c_customer_sk#30] -(82) Sort [codegen id : 31] +(84) Sort [codegen id : 31] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(83) SortMergeJoin [codegen id : 32] +(85) SortMergeJoin [codegen id : 32] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#30] Join condition: None -(84) Project [codegen id : 32] +(86) Project [codegen id : 32] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -(85) HashAggregate [codegen id : 32] +(87) HashAggregate [codegen id : 32] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#55, isEmpty#56] Results [3]: [c_customer_sk#30, sum#57, isEmpty#58] -(86) HashAggregate [codegen id : 32] +(88) HashAggregate [codegen id : 32] Input [3]: [c_customer_sk#30, sum#57, isEmpty#58] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59] Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] -(87) Filter [codegen id : 32] +(89) Filter [codegen id : 32] Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) -(88) Project [codegen id : 32] +(90) Project [codegen id : 32] Output [1]: [c_customer_sk#30] Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] -(89) Sort [codegen id : 32] +(91) Sort [codegen id : 32] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(90) SortMergeJoin +(92) SortMergeJoin Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#30] Join condition: None -(91) Project [codegen id : 34] +(93) Project [codegen id : 34] Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(92) ReusedExchange [Reuses operator id: 57] +(94) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#11] -(93) BroadcastHashJoin [codegen id : 34] +(95) BroadcastHashJoin [codegen id : 34] Left keys [1]: [ws_sold_date_sk#47] Right keys [1]: [d_date_sk#11] Join condition: None -(94) Project [codegen id : 34] +(96) Project [codegen id : 34] Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#61] Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#11] -(95) Union +(97) Union -(96) HashAggregate [codegen id : 35] +(98) HashAggregate [codegen id : 35] Input [1]: [sales#42] Keys: [] Functions [1]: [partial_sum(sales#42)] Aggregate Attributes [2]: [sum#62, isEmpty#63] Results [2]: [sum#64, isEmpty#65] -(97) Exchange +(99) Exchange Input [2]: [sum#64, isEmpty#65] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] -(98) HashAggregate [codegen id : 36] +(100) HashAggregate [codegen id : 36] Input [2]: [sum#64, isEmpty#65] Keys: [] Functions [1]: [sum(sales#42)] @@ -545,47 +557,47 @@ Results [1]: [sum(sales#42)#67 AS sum(sales)#68] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (99) +ReusedExchange (101) -(99) ReusedExchange [Reuses operator id: 57] +(101) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#11] -Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -ReusedExchange (100) +Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 +ReusedExchange (102) -(100) ReusedExchange [Reuses operator id: 12] +(102) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (124) -+- Exchange (123) - +- * HashAggregate (122) - +- * HashAggregate (121) - +- * HashAggregate (120) - +- * Project (119) - +- * SortMergeJoin Inner (118) - :- * Sort (112) - : +- Exchange (111) - : +- * Project (110) - : +- * BroadcastHashJoin Inner BuildRight (109) - : :- * Filter (103) - : : +- * ColumnarToRow (102) - : : +- Scan parquet default.store_sales (101) - : +- BroadcastExchange (108) - : +- * Project (107) - : +- * Filter (106) - : +- * ColumnarToRow (105) - : +- Scan parquet default.date_dim (104) - +- * Sort (117) - +- Exchange (116) - +- * Filter (115) - +- * ColumnarToRow (114) - +- Scan parquet default.customer (113) - - -(101) Scan parquet default.store_sales +Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (126) ++- Exchange (125) + +- * HashAggregate (124) + +- * HashAggregate (123) + +- * HashAggregate (122) + +- * Project (121) + +- * SortMergeJoin Inner (120) + :- * Sort (114) + : +- Exchange (113) + : +- * Project (112) + : +- * BroadcastHashJoin Inner BuildRight (111) + : :- * Filter (105) + : : +- * ColumnarToRow (104) + : : +- Scan parquet default.store_sales (103) + : +- BroadcastExchange (110) + : +- * Project (109) + : +- * Filter (108) + : +- * ColumnarToRow (107) + : +- Scan parquet default.date_dim (106) + +- * Sort (119) + +- Exchange (118) + +- * Filter (117) + +- * ColumnarToRow (116) + +- Scan parquet default.customer (115) + + +(103) Scan parquet default.store_sales Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -593,124 +605,124 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 2] +(104) ColumnarToRow [codegen id : 2] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -(103) Filter [codegen id : 2] +(105) Filter [codegen id : 2] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Condition : isnotnull(ss_customer_sk#26) -(104) Scan parquet default.date_dim +(106) Scan parquet default.date_dim Output [2]: [d_date_sk#11, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(105) ColumnarToRow [codegen id : 1] +(107) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#11, d_year#13] -(106) Filter [codegen id : 1] +(108) Filter [codegen id : 1] Input [2]: [d_date_sk#11, d_year#13] Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) -(107) Project [codegen id : 1] +(109) Project [codegen id : 1] Output [1]: [d_date_sk#11] Input [2]: [d_date_sk#11, d_year#13] -(108) BroadcastExchange +(110) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#70] -(109) BroadcastHashJoin [codegen id : 2] +(111) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join condition: None -(110) Project [codegen id : 2] +(112) Project [codegen id : 2] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9, d_date_sk#11] -(111) Exchange +(113) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#71] -(112) Sort [codegen id : 3] +(114) Sort [codegen id : 3] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 -(113) Scan parquet default.customer +(115) Scan parquet default.customer Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(114) ColumnarToRow [codegen id : 4] +(116) ColumnarToRow [codegen id : 4] Input [1]: [c_customer_sk#30] -(115) Filter [codegen id : 4] +(117) Filter [codegen id : 4] Input [1]: [c_customer_sk#30] Condition : isnotnull(c_customer_sk#30) -(116) Exchange +(118) Exchange Input [1]: [c_customer_sk#30] Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#72] -(117) Sort [codegen id : 5] +(119) Sort [codegen id : 5] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(118) SortMergeJoin [codegen id : 6] +(120) SortMergeJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#30] Join condition: None -(119) Project [codegen id : 6] +(121) Project [codegen id : 6] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -(120) HashAggregate [codegen id : 6] +(122) HashAggregate [codegen id : 6] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#73, isEmpty#74] Results [3]: [c_customer_sk#30, sum#75, isEmpty#76] -(121) HashAggregate [codegen id : 6] +(123) HashAggregate [codegen id : 6] Input [3]: [c_customer_sk#30, sum#75, isEmpty#76] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77] Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77 AS csales#78] -(122) HashAggregate [codegen id : 6] +(124) HashAggregate [codegen id : 6] Input [1]: [csales#78] Keys: [] Functions [1]: [partial_max(csales#78)] Aggregate Attributes [1]: [max#79] Results [1]: [max#80] -(123) Exchange +(125) Exchange Input [1]: [max#80] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] -(124) HashAggregate [codegen id : 7] +(126) HashAggregate [codegen id : 7] Input [1]: [max#80] Keys: [] Functions [1]: [max(csales#78)] Aggregate Attributes [1]: [max(csales#78)#82] Results [1]: [max(csales#78)#82 AS tpcds_cmax#83] -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#69 -ReusedExchange (125) +Subquery:4 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#69 +ReusedExchange (127) -(125) ReusedExchange [Reuses operator id: 108] +(127) ReusedExchange [Reuses operator id: 110] Output [1]: [d_date_sk#11] -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:6 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt index 1e3e8aaa3cb96..e48dc68bce924 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt @@ -25,11 +25,12 @@ WholeStageCodegen (36) InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (1) - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 + Filter [cs_item_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 WholeStageCodegen (8) Sort [item_sk] Project [item_sk] @@ -169,10 +170,11 @@ WholeStageCodegen (36) InputAdapter Exchange [ws_item_sk] #15 WholeStageCodegen (18) - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_item_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 WholeStageCodegen (25) Sort [item_sk] Project [item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index 692671c9f287d..e3f996ad72e41 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -1,78 +1,80 @@ == Physical Plan == -* HashAggregate (74) -+- Exchange (73) - +- * HashAggregate (72) - +- Union (71) - :- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) - : :- * Project (46) - : : +- SortMergeJoin LeftSemi (45) - : : :- * Sort (28) - : : : +- Exchange (27) - : : : +- * Project (26) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (25) - : : : :- * ColumnarToRow (2) - : : : : +- Scan parquet default.catalog_sales (1) - : : : +- BroadcastExchange (24) - : : : +- * Project (23) - : : : +- * Filter (22) - : : : +- * HashAggregate (21) - : : : +- Exchange (20) - : : : +- * HashAggregate (19) - : : : +- * Project (18) - : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : :- * Project (12) - : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : :- * Filter (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- Scan parquet default.store_sales (3) - : : : : +- BroadcastExchange (10) - : : : : +- * Project (9) - : : : : +- * Filter (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.date_dim (6) - : : : +- BroadcastExchange (16) - : : : +- * Filter (15) - : : : +- * ColumnarToRow (14) - : : : +- Scan parquet default.item (13) - : : +- * Sort (44) - : : +- * Project (43) - : : +- * Filter (42) - : : +- * HashAggregate (41) - : : +- Exchange (40) - : : +- * HashAggregate (39) - : : +- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (32) - : : : +- * Filter (31) - : : : +- * ColumnarToRow (30) - : : : +- Scan parquet default.store_sales (29) - : : +- BroadcastExchange (36) - : : +- * Filter (35) - : : +- * ColumnarToRow (34) - : : +- Scan parquet default.customer (33) - : +- BroadcastExchange (51) - : +- * Project (50) - : +- * Filter (49) - : +- * ColumnarToRow (48) - : +- Scan parquet default.date_dim (47) - +- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (67) - : +- SortMergeJoin LeftSemi (66) - : :- * Sort (60) - : : +- Exchange (59) - : : +- * Project (58) - : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * ColumnarToRow (55) - : : : +- Scan parquet default.web_sales (54) - : : +- ReusedExchange (56) - : +- * Sort (65) - : +- * Project (64) - : +- * Filter (63) - : +- * HashAggregate (62) - : +- ReusedExchange (61) - +- ReusedExchange (68) +* HashAggregate (76) ++- Exchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (54) + : +- * BroadcastHashJoin Inner BuildRight (53) + : :- * Project (47) + : : +- SortMergeJoin LeftSemi (46) + : : :- * Sort (29) + : : : +- Exchange (28) + : : : +- * Project (27) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.catalog_sales (1) + : : : +- BroadcastExchange (25) + : : : +- * Project (24) + : : : +- * Filter (23) + : : : +- * HashAggregate (22) + : : : +- Exchange (21) + : : : +- * HashAggregate (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * Project (13) + : : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.store_sales (4) + : : : : +- BroadcastExchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.date_dim (7) + : : : +- BroadcastExchange (17) + : : : +- * Filter (16) + : : : +- * ColumnarToRow (15) + : : : +- Scan parquet default.item (14) + : : +- * Sort (45) + : : +- * Project (44) + : : +- * Filter (43) + : : +- * HashAggregate (42) + : : +- Exchange (41) + : : +- * HashAggregate (40) + : : +- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * Project (33) + : : : +- * Filter (32) + : : : +- * ColumnarToRow (31) + : : : +- Scan parquet default.store_sales (30) + : : +- BroadcastExchange (37) + : : +- * Filter (36) + : : +- * ColumnarToRow (35) + : : +- Scan parquet default.customer (34) + : +- BroadcastExchange (52) + : +- * Project (51) + : +- * Filter (50) + : +- * ColumnarToRow (49) + : +- Scan parquet default.date_dim (48) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- * Project (69) + : +- SortMergeJoin LeftSemi (68) + : :- * Sort (62) + : : +- Exchange (61) + : : +- * Project (60) + : : +- * BroadcastHashJoin LeftSemi BuildRight (59) + : : :- * Filter (57) + : : : +- * ColumnarToRow (56) + : : : +- Scan parquet default.web_sales (55) + : : +- ReusedExchange (58) + : +- * Sort (67) + : +- * Project (66) + : +- * Filter (65) + : +- * HashAggregate (64) + : +- ReusedExchange (63) + +- ReusedExchange (70) (1) Scan parquet default.catalog_sales @@ -80,12 +82,17 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(3) Scan parquet default.store_sales +(3) Filter [codegen id : 5] +Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) + +(4) Scan parquet default.store_sales Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -93,320 +100,325 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(4) ColumnarToRow [codegen id : 3] +(5) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(5) Filter [codegen id : 3] +(6) Filter [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(6) Scan parquet default.date_dim +(7) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_date#11, d_year#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 1] +(8) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#12] -(8) Filter [codegen id : 1] +(9) Filter [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#12] Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(9) Project [codegen id : 1] +(10) Project [codegen id : 1] Output [2]: [d_date_sk#10, d_date#11] Input [3]: [d_date_sk#10, d_date#11, d_year#12] -(10) BroadcastExchange +(11) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(11) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(12) Project [codegen id : 3] +(13) Project [codegen id : 3] Output [2]: [ss_item_sk#7, d_date#11] Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -(13) Scan parquet default.item +(14) Scan parquet default.item Output [2]: [i_item_sk#14, i_item_desc#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(14) ColumnarToRow [codegen id : 2] +(15) ColumnarToRow [codegen id : 2] Input [2]: [i_item_sk#14, i_item_desc#15] -(15) Filter [codegen id : 2] +(16) Filter [codegen id : 2] Input [2]: [i_item_sk#14, i_item_desc#15] Condition : isnotnull(i_item_sk#14) -(16) BroadcastExchange +(17) BroadcastExchange Input [2]: [i_item_sk#14, i_item_desc#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] -(17) BroadcastHashJoin [codegen id : 3] +(18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#7] Right keys [1]: [i_item_sk#14] Join condition: None -(18) Project [codegen id : 3] +(19) Project [codegen id : 3] Output [3]: [d_date#11, i_item_sk#14, i_item_desc#15] Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#14, i_item_desc#15] -(19) HashAggregate [codegen id : 3] +(20) HashAggregate [codegen id : 3] Input [3]: [d_date#11, i_item_sk#14, i_item_desc#15] Keys [3]: [substr(i_item_desc#15, 1, 30) AS substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#18] Results [4]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11, count#19] -(20) Exchange +(21) Exchange Input [4]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11, count#19] Arguments: hashpartitioning(substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11, 5), ENSURE_REQUIREMENTS, [id=#20] -(21) HashAggregate [codegen id : 4] +(22) HashAggregate [codegen id : 4] Input [4]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11, count#19] Keys [3]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#21] Results [2]: [i_item_sk#14 AS item_sk#22, count(1)#21 AS count(1)#23] -(22) Filter [codegen id : 4] +(23) Filter [codegen id : 4] Input [2]: [item_sk#22, count(1)#23] Condition : (count(1)#23 > 4) -(23) Project [codegen id : 4] +(24) Project [codegen id : 4] Output [1]: [item_sk#22] Input [2]: [item_sk#22, count(1)#23] -(24) BroadcastExchange +(25) BroadcastExchange Input [1]: [item_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(25) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#22] Join condition: None -(26) Project [codegen id : 5] +(27) Project [codegen id : 5] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(27) Exchange +(28) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#25] -(28) Sort [codegen id : 6] +(29) Sort [codegen id : 6] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(29) Scan parquet default.store_sales +(30) Scan parquet default.store_sales Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 8] +(31) ColumnarToRow [codegen id : 8] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -(31) Filter [codegen id : 8] +(32) Filter [codegen id : 8] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Condition : isnotnull(ss_customer_sk#26) -(32) Project [codegen id : 8] +(33) Project [codegen id : 8] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -(33) Scan parquet default.customer +(34) Scan parquet default.customer Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 7] +(35) ColumnarToRow [codegen id : 7] Input [1]: [c_customer_sk#29] -(35) Filter [codegen id : 7] +(36) Filter [codegen id : 7] Input [1]: [c_customer_sk#29] Condition : isnotnull(c_customer_sk#29) -(36) BroadcastExchange +(37) BroadcastExchange Input [1]: [c_customer_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] -(37) BroadcastHashJoin [codegen id : 8] +(38) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#29] Join condition: None -(38) Project [codegen id : 8] +(39) Project [codegen id : 8] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -(39) HashAggregate [codegen id : 8] +(40) HashAggregate [codegen id : 8] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] Keys [1]: [c_customer_sk#29] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#31, isEmpty#32] Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] -(40) Exchange +(41) Exchange Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#35] -(41) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 9] Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -(42) Filter [codegen id : 9] +(43) Filter [codegen id : 9] Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) -(43) Project [codegen id : 9] +(44) Project [codegen id : 9] Output [1]: [c_customer_sk#29] Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -(44) Sort [codegen id : 9] +(45) Sort [codegen id : 9] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(45) SortMergeJoin +(46) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#29] Join condition: None -(46) Project [codegen id : 11] +(47) Project [codegen id : 11] Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(47) Scan parquet default.date_dim +(48) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#12, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 10] +(49) ColumnarToRow [codegen id : 10] Input [3]: [d_date_sk#10, d_year#12, d_moy#40] -(49) Filter [codegen id : 10] +(50) Filter [codegen id : 10] Input [3]: [d_date_sk#10, d_year#12, d_moy#40] Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#40)) AND (d_year#12 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#10)) -(50) Project [codegen id : 10] +(51) Project [codegen id : 10] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#12, d_moy#40] -(51) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] -(52) BroadcastHashJoin [codegen id : 11] +(53) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#10] Join condition: None -(53) Project [codegen id : 11] +(54) Project [codegen id : 11] Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#10] -(54) Scan parquet default.web_sales +(55) Scan parquet default.web_sales Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 16] +(56) ColumnarToRow [codegen id : 16] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] + +(57) Filter [codegen id : 16] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Condition : (isnotnull(ws_item_sk#43) AND isnotnull(ws_bill_customer_sk#44)) -(56) ReusedExchange [Reuses operator id: unknown] +(58) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] -(57) BroadcastHashJoin [codegen id : 16] +(59) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#22] Join condition: None -(58) Project [codegen id : 16] +(60) Project [codegen id : 16] Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(59) Exchange +(61) Exchange Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#48] -(60) Sort [codegen id : 17] +(62) Sort [codegen id : 17] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 -(61) ReusedExchange [Reuses operator id: 40] +(63) ReusedExchange [Reuses operator id: 41] Output [3]: [c_customer_sk#29, sum#49, isEmpty#50] -(62) HashAggregate [codegen id : 20] +(64) HashAggregate [codegen id : 20] Input [3]: [c_customer_sk#29, sum#49, isEmpty#50] Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51] Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] -(63) Filter [codegen id : 20] +(65) Filter [codegen id : 20] Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) -(64) Project [codegen id : 20] +(66) Project [codegen id : 20] Output [1]: [c_customer_sk#29] Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] -(65) Sort [codegen id : 20] +(67) Sort [codegen id : 20] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(66) SortMergeJoin +(68) SortMergeJoin Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#29] Join condition: None -(67) Project [codegen id : 22] +(69) Project [codegen id : 22] Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(68) ReusedExchange [Reuses operator id: 51] +(70) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#10] -(69) BroadcastHashJoin [codegen id : 22] +(71) BroadcastHashJoin [codegen id : 22] Left keys [1]: [ws_sold_date_sk#47] Right keys [1]: [d_date_sk#10] Join condition: None -(70) Project [codegen id : 22] +(72) Project [codegen id : 22] Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#53] Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#10] -(71) Union +(73) Union -(72) HashAggregate [codegen id : 23] +(74) HashAggregate [codegen id : 23] Input [1]: [sales#42] Keys: [] Functions [1]: [partial_sum(sales#42)] Aggregate Attributes [2]: [sum#54, isEmpty#55] Results [2]: [sum#56, isEmpty#57] -(73) Exchange +(75) Exchange Input [2]: [sum#56, isEmpty#57] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#58] -(74) HashAggregate [codegen id : 24] +(76) HashAggregate [codegen id : 24] Input [2]: [sum#56, isEmpty#57] Keys: [] Functions [1]: [sum(sales#42)] @@ -416,45 +428,45 @@ Results [1]: [sum(sales#42)#59 AS sum(sales)#60] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (75) +ReusedExchange (77) -(75) ReusedExchange [Reuses operator id: 51] +(77) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (76) +Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +ReusedExchange (78) -(76) ReusedExchange [Reuses operator id: 10] +(78) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (98) -+- Exchange (97) - +- * HashAggregate (96) - +- * HashAggregate (95) - +- Exchange (94) - +- * HashAggregate (93) - +- * Project (92) - +- * BroadcastHashJoin Inner BuildRight (91) - :- * Project (85) - : +- * BroadcastHashJoin Inner BuildRight (84) - : :- * Filter (79) - : : +- * ColumnarToRow (78) - : : +- Scan parquet default.store_sales (77) - : +- BroadcastExchange (83) - : +- * Filter (82) - : +- * ColumnarToRow (81) - : +- Scan parquet default.customer (80) - +- BroadcastExchange (90) - +- * Project (89) - +- * Filter (88) - +- * ColumnarToRow (87) - +- Scan parquet default.date_dim (86) - - -(77) Scan parquet default.store_sales +Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (100) ++- Exchange (99) + +- * HashAggregate (98) + +- * HashAggregate (97) + +- Exchange (96) + +- * HashAggregate (95) + +- * Project (94) + +- * BroadcastHashJoin Inner BuildRight (93) + :- * Project (87) + : +- * BroadcastHashJoin Inner BuildRight (86) + : :- * Filter (81) + : : +- * ColumnarToRow (80) + : : +- Scan parquet default.store_sales (79) + : +- BroadcastExchange (85) + : +- * Filter (84) + : +- * ColumnarToRow (83) + : +- Scan parquet default.customer (82) + +- BroadcastExchange (92) + +- * Project (91) + +- * Filter (90) + +- * ColumnarToRow (89) + +- Scan parquet default.date_dim (88) + + +(79) Scan parquet default.store_sales Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -462,116 +474,116 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(78) ColumnarToRow [codegen id : 3] +(80) ColumnarToRow [codegen id : 3] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -(79) Filter [codegen id : 3] +(81) Filter [codegen id : 3] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Condition : isnotnull(ss_customer_sk#26) -(80) Scan parquet default.customer +(82) Scan parquet default.customer Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(81) ColumnarToRow [codegen id : 1] +(83) ColumnarToRow [codegen id : 1] Input [1]: [c_customer_sk#29] -(82) Filter [codegen id : 1] +(84) Filter [codegen id : 1] Input [1]: [c_customer_sk#29] Condition : isnotnull(c_customer_sk#29) -(83) BroadcastExchange +(85) BroadcastExchange Input [1]: [c_customer_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#62] -(84) BroadcastHashJoin [codegen id : 3] +(86) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#29] Join condition: None -(85) Project [codegen id : 3] +(87) Project [codegen id : 3] Output [4]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] -(86) Scan parquet default.date_dim +(88) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(87) ColumnarToRow [codegen id : 2] +(89) ColumnarToRow [codegen id : 2] Input [2]: [d_date_sk#10, d_year#12] -(88) Filter [codegen id : 2] +(90) Filter [codegen id : 2] Input [2]: [d_date_sk#10, d_year#12] Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(89) Project [codegen id : 2] +(91) Project [codegen id : 2] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#12] -(90) BroadcastExchange +(92) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] -(91) BroadcastHashJoin [codegen id : 3] +(93) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(92) Project [codegen id : 3] +(94) Project [codegen id : 3] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] Input [5]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29, d_date_sk#10] -(93) HashAggregate [codegen id : 3] +(95) HashAggregate [codegen id : 3] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] Keys [1]: [c_customer_sk#29] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#64, isEmpty#65] Results [3]: [c_customer_sk#29, sum#66, isEmpty#67] -(94) Exchange +(96) Exchange Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#68] -(95) HashAggregate [codegen id : 4] +(97) HashAggregate [codegen id : 4] Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS csales#70] -(96) HashAggregate [codegen id : 4] +(98) HashAggregate [codegen id : 4] Input [1]: [csales#70] Keys: [] Functions [1]: [partial_max(csales#70)] Aggregate Attributes [1]: [max#71] Results [1]: [max#72] -(97) Exchange +(99) Exchange Input [1]: [max#72] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#73] -(98) HashAggregate [codegen id : 5] +(100) HashAggregate [codegen id : 5] Input [1]: [max#72] Keys: [] Functions [1]: [max(csales#70)] Aggregate Attributes [1]: [max(csales#70)#74] Results [1]: [max(csales#70)#74 AS tpcds_cmax#75] -Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#61 -ReusedExchange (99) +Subquery:4 Hosting operator id = 79 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#61 +ReusedExchange (101) -(99) ReusedExchange [Reuses operator id: 90] +(101) ReusedExchange [Reuses operator id: 92] Output [1]: [d_date_sk#10] -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 55 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:6 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index 87f135b3fff0d..36bc26bc541bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -19,11 +19,12 @@ WholeStageCodegen (24) WholeStageCodegen (5) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + Filter [cs_item_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 WholeStageCodegen (4) @@ -140,10 +141,11 @@ WholeStageCodegen (24) WholeStageCodegen (16) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_item_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [item_sk] #15 WholeStageCodegen (20) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt index 35a42fa256325..0ac39b5242dce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt @@ -143,7 +143,7 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] +PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -151,7 +151,7 @@ Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, (3) Filter [codegen id : 1] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_bill_customer_sk#1) +Condition : (isnotnull(cs_bill_customer_sk#1) AND isnotnull(cs_item_sk#2)) (4) Exchange Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] @@ -516,7 +516,7 @@ Output [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_pric Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] +PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (83) ColumnarToRow [codegen id : 27] @@ -524,7 +524,7 @@ Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price (84) Filter [codegen id : 27] Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Condition : isnotnull(ws_bill_customer_sk#54) +Condition : (isnotnull(ws_bill_customer_sk#54) AND isnotnull(ws_item_sk#53)) (85) Exchange Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt index 1962a53fa34c2..110e3e50157ba 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt @@ -27,7 +27,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (1) - Filter [cs_bill_customer_sk] + Filter [cs_bill_customer_sk,cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] @@ -210,7 +210,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter Exchange [ws_item_sk] #17 WholeStageCodegen (27) - Filter [ws_bill_customer_sk] + Filter [ws_bill_customer_sk,ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 385cf017e58c3..7b4a41ff0813f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -109,7 +109,7 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] +PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 5] @@ -117,7 +117,7 @@ Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, (3) Filter [codegen id : 5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_bill_customer_sk#1) +Condition : (isnotnull(cs_bill_customer_sk#1) AND isnotnull(cs_item_sk#2)) (4) Scan parquet default.store_sales Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] @@ -435,7 +435,7 @@ Output [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_pric Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] +PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 22] @@ -443,7 +443,7 @@ Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price (73) Filter [codegen id : 22] Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_customer_sk#55) +Condition : (isnotnull(ws_bill_customer_sk#55) AND isnotnull(ws_item_sk#54)) (74) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index a18328434b618..e405cfaa0d9bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_bill_customer_sk] + Filter [cs_bill_customer_sk,cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] @@ -167,7 +167,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (22) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_bill_customer_sk] + Filter [ws_bill_customer_sk,ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index cba099c2bb5b0..aff4892e0231e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_manufact_id)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_manufact_id#14] -Condition : isnotnull(i_item_sk#13) +Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_manufact_id#14)) (21) Scan parquet default.item Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics ), IsNotNull(i_manufact_id)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) +Condition : ((isnotnull(i_category#15) AND (i_category#15 = Electronics )) AND isnotnull(i_manufact_id#14)) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index d1a115fa528d5..6a4f8b5362c74 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] + Filter [i_item_sk,i_manufact_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #6 WholeStageCodegen (3) Project [i_manufact_id] - Filter [i_category] + Filter [i_category,i_manufact_id] ColumnarToRow InputAdapter Scan parquet default.item [i_category,i_manufact_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index cba099c2bb5b0..aff4892e0231e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_manufact_id)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_manufact_id#14] -Condition : isnotnull(i_item_sk#13) +Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_manufact_id#14)) (21) Scan parquet default.item Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics ), IsNotNull(i_manufact_id)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) +Condition : ((isnotnull(i_category#15) AND (i_category#15 = Electronics )) AND isnotnull(i_manufact_id#14)) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index 7098389df6776..82127815c5d69 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -1,68 +1,69 @@ == Physical Plan == -TakeOrderedAndProject (57) -+- * HashAggregate (56) - +- Exchange (55) - +- * HashAggregate (54) - +- * Project (53) - +- * SortMergeJoin Inner (52) - :- * Sort (46) - : +- Exchange (45) - : +- * Project (44) - : +- * SortMergeJoin Inner (43) - : :- * Sort (37) - : : +- Exchange (36) - : : +- * Project (35) - : : +- * Filter (34) - : : +- SortMergeJoin ExistenceJoin(exists#1) (33) - : : :- SortMergeJoin ExistenceJoin(exists#2) (25) - : : : :- SortMergeJoin LeftSemi (17) +TakeOrderedAndProject (58) ++- * HashAggregate (57) + +- Exchange (56) + +- * HashAggregate (55) + +- * Project (54) + +- * SortMergeJoin Inner (53) + :- * Sort (47) + : +- Exchange (46) + : +- * Project (45) + : +- * SortMergeJoin Inner (44) + : :- * Sort (38) + : : +- Exchange (37) + : : +- * Project (36) + : : +- * Filter (35) + : : +- SortMergeJoin ExistenceJoin(exists#1) (34) + : : :- SortMergeJoin ExistenceJoin(exists#2) (26) + : : : :- SortMergeJoin LeftSemi (18) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- * Sort (16) - : : : : +- Exchange (15) - : : : : +- * Project (14) - : : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : : :- * ColumnarToRow (7) - : : : : : +- Scan parquet default.store_sales (6) - : : : : +- BroadcastExchange (12) - : : : : +- * Project (11) - : : : : +- * Filter (10) - : : : : +- * ColumnarToRow (9) - : : : : +- Scan parquet default.date_dim (8) - : : : +- * Sort (24) - : : : +- Exchange (23) - : : : +- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * ColumnarToRow (19) - : : : : +- Scan parquet default.web_sales (18) - : : : +- ReusedExchange (20) - : : +- * Sort (32) - : : +- Exchange (31) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * ColumnarToRow (27) - : : : +- Scan parquet default.catalog_sales (26) - : : +- ReusedExchange (28) - : +- * Sort (42) - : +- Exchange (41) - : +- * Filter (40) - : +- * ColumnarToRow (39) - : +- Scan parquet default.customer_address (38) - +- * Sort (51) - +- Exchange (50) - +- * Filter (49) - +- * ColumnarToRow (48) - +- Scan parquet default.customer_demographics (47) + : : : : +- * Sort (17) + : : : : +- Exchange (16) + : : : : +- * Project (15) + : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * Filter (11) + : : : : +- * ColumnarToRow (10) + : : : : +- Scan parquet default.date_dim (9) + : : : +- * Sort (25) + : : : +- Exchange (24) + : : : +- * Project (23) + : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : :- * ColumnarToRow (20) + : : : : +- Scan parquet default.web_sales (19) + : : : +- ReusedExchange (21) + : : +- * Sort (33) + : : +- Exchange (32) + : : +- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * ColumnarToRow (28) + : : : +- Scan parquet default.catalog_sales (27) + : : +- ReusedExchange (29) + : +- * Sort (43) + : +- Exchange (42) + : +- * Filter (41) + : +- * ColumnarToRow (40) + : +- Scan parquet default.customer_address (39) + +- * Sort (52) + +- Exchange (51) + +- * Filter (50) + +- * ColumnarToRow (49) + +- Scan parquet default.customer_demographics (48) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -70,7 +71,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -85,244 +86,249 @@ Output [2]: [ss_customer_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -(8) Scan parquet default.date_dim +(8) Filter [codegen id : 4] +Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] +Condition : isnotnull(ss_customer_sk#7) + +(9) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_qoy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(9) ColumnarToRow [codegen id : 3] +(10) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] -(10) Filter [codegen id : 3] +(11) Filter [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_qoy#12)) AND (d_year#11 = 2002)) AND (d_qoy#12 < 4)) AND isnotnull(d_date_sk#10)) -(11) Project [codegen id : 3] +(12) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] -(12) BroadcastExchange +(13) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(13) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(14) Project [codegen id : 4] +(15) Project [codegen id : 4] Output [1]: [ss_customer_sk#7] Input [3]: [ss_customer_sk#7, ss_sold_date_sk#8, d_date_sk#10] -(15) Exchange +(16) Exchange Input [1]: [ss_customer_sk#7] Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] -(16) Sort [codegen id : 5] +(17) Sort [codegen id : 5] Input [1]: [ss_customer_sk#7] Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin +(18) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#7] Join condition: None -(18) Scan parquet default.web_sales +(19) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#16), dynamicpruningexpression(ws_sold_date_sk#16 IN dynamicpruning#9)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(20) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] -(20) ReusedExchange [Reuses operator id: 12] +(21) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#10] -(21) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(22) Project [codegen id : 7] +(23) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] -(23) Exchange +(24) Exchange Input [1]: [ws_bill_customer_sk#15] Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] -(24) Sort [codegen id : 8] +(25) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] Arguments: [ws_bill_customer_sk#15 ASC NULLS FIRST], false, 0 -(25) SortMergeJoin +(26) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#15] Join condition: None -(26) Scan parquet default.catalog_sales +(27) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 10] +(28) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] -(28) ReusedExchange [Reuses operator id: 12] +(29) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#10] -(29) BroadcastHashJoin [codegen id : 10] +(30) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#19] Right keys [1]: [d_date_sk#10] Join condition: None -(30) Project [codegen id : 10] +(31) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#18] Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] -(31) Exchange +(32) Exchange Input [1]: [cs_ship_customer_sk#18] Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] -(32) Sort [codegen id : 11] +(33) Sort [codegen id : 11] Input [1]: [cs_ship_customer_sk#18] Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 -(33) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#18] Join condition: None -(34) Filter [codegen id : 12] +(35) Filter [codegen id : 12] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(35) Project [codegen id : 12] +(36) Project [codegen id : 12] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(36) Exchange +(37) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] -(37) Sort [codegen id : 13] +(38) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.customer_address +(39) Scan parquet default.customer_address Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 14] +(40) ColumnarToRow [codegen id : 14] Input [2]: [ca_address_sk#22, ca_state#23] -(40) Filter [codegen id : 14] +(41) Filter [codegen id : 14] Input [2]: [ca_address_sk#22, ca_state#23] Condition : isnotnull(ca_address_sk#22) -(41) Exchange +(42) Exchange Input [2]: [ca_address_sk#22, ca_state#23] Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] -(42) Sort [codegen id : 15] +(43) Sort [codegen id : 15] Input [2]: [ca_address_sk#22, ca_state#23] Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 -(43) SortMergeJoin [codegen id : 16] +(44) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#22] Join condition: None -(44) Project [codegen id : 16] +(45) Project [codegen id : 16] Output [2]: [c_current_cdemo_sk#4, ca_state#23] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] -(45) Exchange +(46) Exchange Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] -(46) Sort [codegen id : 17] +(47) Sort [codegen id : 17] Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(47) Scan parquet default.customer_demographics +(48) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 18] +(49) ColumnarToRow [codegen id : 18] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(49) Filter [codegen id : 18] +(50) Filter [codegen id : 18] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Condition : isnotnull(cd_demo_sk#26) -(50) Exchange +(51) Exchange Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] -(51) Sort [codegen id : 19] +(52) Sort [codegen id : 19] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 -(52) SortMergeJoin [codegen id : 20] +(53) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#26] Join condition: None -(53) Project [codegen id : 20] +(54) Project [codegen id : 20] Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(54) HashAggregate [codegen id : 20] +(55) HashAggregate [codegen id : 20] Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [10]: [partial_count(1), partial_min(cd_dep_count#29), partial_max(cd_dep_count#29), partial_avg(cd_dep_count#29), partial_min(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_avg(cd_dep_employed_count#30), partial_min(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_avg(cd_dep_college_count#31)] Aggregate Attributes [13]: [count#33, min#34, max#35, sum#36, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45] Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] -(55) Exchange +(56) Exchange Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] -(56) HashAggregate [codegen id : 21] +(57) HashAggregate [codegen id : 21] Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [10]: [count(1), min(cd_dep_count#29), max(cd_dep_count#29), avg(cd_dep_count#29), min(cd_dep_employed_count#30), max(cd_dep_employed_count#30), avg(cd_dep_employed_count#30), min(cd_dep_college_count#31), max(cd_dep_college_count#31), avg(cd_dep_college_count#31)] Aggregate Attributes [10]: [count(1)#60, min(cd_dep_count#29)#61, max(cd_dep_count#29)#62, avg(cd_dep_count#29)#63, min(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, avg(cd_dep_employed_count#30)#66, min(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, avg(cd_dep_college_count#31)#69] Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, count(1)#60 AS cnt1#70, min(cd_dep_count#29)#61 AS min(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, avg(cd_dep_count#29)#63 AS avg(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, min(cd_dep_employed_count#30)#64 AS min(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, avg(cd_dep_employed_count#30)#66 AS avg(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, min(cd_dep_college_count#31)#67 AS min(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, avg(cd_dep_college_count#31)#69 AS avg(cd_dep_college_count)#81, cd_dep_count#29 AS aggOrder#82] -(57) TakeOrderedAndProject +(58) TakeOrderedAndProject Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81, aggOrder#82] Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, aggOrder#82 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (58) +ReusedExchange (59) -(58) ReusedExchange [Reuses operator id: 12] +(59) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt index 6c399f1bd2197..60624e1380e63 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -43,11 +43,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #6 + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #6 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 4dab13491784a..661cac7ce3ef0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -1,57 +1,58 @@ == Physical Plan == -TakeOrderedAndProject (46) -+- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (30) - : : +- * Filter (29) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (28) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (21) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (14) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (31) + : : +- * Filter (30) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (29) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (22) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (13) - : : : : +- * Project (12) - : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (10) - : : : : +- * Project (9) - : : : : +- * Filter (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.date_dim (6) - : : : +- BroadcastExchange (20) - : : : +- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * ColumnarToRow (16) - : : : : +- Scan parquet default.web_sales (15) - : : : +- ReusedExchange (17) - : : +- BroadcastExchange (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (24) - : +- BroadcastExchange (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.customer_address (31) - +- BroadcastExchange (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet default.customer_demographics (37) + : : : : +- BroadcastExchange (14) + : : : : +- * Project (13) + : : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.store_sales (4) + : : : : +- BroadcastExchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.date_dim (7) + : : : +- BroadcastExchange (21) + : : : +- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * ColumnarToRow (17) + : : : : +- Scan parquet default.web_sales (16) + : : : +- ReusedExchange (18) + : : +- BroadcastExchange (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * ColumnarToRow (24) + : : : +- Scan parquet default.catalog_sales (23) + : : +- ReusedExchange (25) + : +- BroadcastExchange (35) + : +- * Filter (34) + : +- * ColumnarToRow (33) + : +- Scan parquet default.customer_address (32) + +- BroadcastExchange (41) + +- * Filter (40) + +- * ColumnarToRow (39) + +- Scan parquet default.customer_demographics (38) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -59,215 +60,220 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) Scan parquet default.date_dim +(6) Filter [codegen id : 2] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#6) + +(7) Scan parquet default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 1] +(8) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -(8) Filter [codegen id : 1] +(9) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(9) Project [codegen id : 1] +(10) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -(10) BroadcastExchange +(11) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] -(11) BroadcastHashJoin [codegen id : 2] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join condition: None -(12) Project [codegen id : 2] +(13) Project [codegen id : 2] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(13) BroadcastExchange +(14) BroadcastExchange Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(14) BroadcastHashJoin [codegen id : 9] +(15) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join condition: None -(15) Scan parquet default.web_sales +(16) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#15), dynamicpruningexpression(ws_sold_date_sk#15 IN dynamicpruning#8)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 4] +(17) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] -(17) ReusedExchange [Reuses operator id: 10] +(18) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#9] -(18) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] Right keys [1]: [d_date_sk#9] Join condition: None -(19) Project [codegen id : 4] +(20) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] -(20) BroadcastExchange +(21) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(21) BroadcastHashJoin [codegen id : 9] +(22) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#14] Join condition: None -(22) Scan parquet default.catalog_sales +(23) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 6] +(24) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] -(24) ReusedExchange [Reuses operator id: 10] +(25) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#9] -(25) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#9] Join condition: None -(26) Project [codegen id : 6] +(27) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#17] Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] -(27) BroadcastExchange +(28) BroadcastExchange Input [1]: [cs_ship_customer_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(28) BroadcastHashJoin [codegen id : 9] +(29) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#17] Join condition: None -(29) Filter [codegen id : 9] +(30) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(30) Project [codegen id : 9] +(31) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(31) Scan parquet default.customer_address +(32) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 7] +(33) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#20, ca_state#21] -(33) Filter [codegen id : 7] +(34) Filter [codegen id : 7] Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(34) BroadcastExchange +(35) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(35) BroadcastHashJoin [codegen id : 9] +(36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join condition: None -(36) Project [codegen id : 9] +(37) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, ca_state#21] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] -(37) Scan parquet default.customer_demographics +(38) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(38) ColumnarToRow [codegen id : 8] +(39) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(39) Filter [codegen id : 8] +(40) Filter [codegen id : 8] Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Condition : isnotnull(cd_demo_sk#23) -(40) BroadcastExchange +(41) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(41) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join condition: None -(42) Project [codegen id : 9] +(43) Project [codegen id : 9] Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) HashAggregate [codegen id : 9] +(44) HashAggregate [codegen id : 9] Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] Aggregate Attributes [13]: [count#30, min#31, max#32, sum#33, count#34, min#35, max#36, sum#37, count#38, min#39, max#40, sum#41, count#42] Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] -(44) Exchange +(45) Exchange Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] -(45) HashAggregate [codegen id : 10] +(46) HashAggregate [codegen id : 10] Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] Aggregate Attributes [10]: [count(1)#57, min(cd_dep_count#26)#58, max(cd_dep_count#26)#59, avg(cd_dep_count#26)#60, min(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, avg(cd_dep_employed_count#27)#63, min(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, avg(cd_dep_college_count#28)#66] Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, count(1)#57 AS cnt1#67, min(cd_dep_count#26)#58 AS min(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, avg(cd_dep_count#26)#60 AS avg(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, min(cd_dep_employed_count#27)#61 AS min(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, avg(cd_dep_employed_count#27)#63 AS avg(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, min(cd_dep_college_count#28)#64 AS min(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, avg(cd_dep_college_count#28)#66 AS avg(cd_dep_college_count)#78, cd_dep_count#26 AS aggOrder#79] -(46) TakeOrderedAndProject +(47) TakeOrderedAndProject Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78, aggOrder#79] Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, aggOrder#79 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -ReusedExchange (47) +ReusedExchange (48) -(47) ReusedExchange [Reuses operator id: 10] +(48) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#9] -Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index 2614f4f8ae881..4010c2656300c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -23,11 +23,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index 2d3dea5d84565..30ecc62144ddb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (81) -+- * HashAggregate (80) - +- Exchange (79) - +- * HashAggregate (78) - +- * Expand (77) - +- Union (76) +TakeOrderedAndProject (82) ++- * HashAggregate (81) + +- Exchange (80) + +- * HashAggregate (79) + +- * Expand (78) + +- Union (77) :- * HashAggregate (25) : +- Exchange (24) : +- * HashAggregate (23) @@ -51,35 +51,36 @@ TakeOrderedAndProject (81) : : +- * ColumnarToRow (36) : : +- Scan parquet default.catalog_page (35) : +- ReusedExchange (41) - +- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (69) - : +- * BroadcastHashJoin Inner BuildRight (68) - : :- Union (63) + +- * HashAggregate (76) + +- Exchange (75) + +- * HashAggregate (74) + +- * Project (73) + +- * BroadcastHashJoin Inner BuildRight (72) + :- * Project (70) + : +- * BroadcastHashJoin Inner BuildRight (69) + : :- Union (64) : : :- * Project (50) : : : +- * Filter (49) : : : +- * ColumnarToRow (48) : : : +- Scan parquet default.web_sales (47) - : : +- * Project (62) - : : +- * SortMergeJoin Inner (61) - : : :- * Sort (54) - : : : +- Exchange (53) - : : : +- * ColumnarToRow (52) - : : : +- Scan parquet default.web_returns (51) - : : +- * Sort (60) - : : +- Exchange (59) - : : +- * Project (58) - : : +- * Filter (57) - : : +- * ColumnarToRow (56) - : : +- Scan parquet default.web_sales (55) - : +- BroadcastExchange (67) - : +- * Filter (66) - : +- * ColumnarToRow (65) - : +- Scan parquet default.web_site (64) - +- ReusedExchange (70) + : : +- * Project (63) + : : +- * SortMergeJoin Inner (62) + : : :- * Sort (55) + : : : +- Exchange (54) + : : : +- * Filter (53) + : : : +- * ColumnarToRow (52) + : : : +- Scan parquet default.web_returns (51) + : : +- * Sort (61) + : : +- Exchange (60) + : : +- * Project (59) + : : +- * Filter (58) + : : +- * ColumnarToRow (57) + : : +- Scan parquet default.web_sales (56) + : +- BroadcastExchange (68) + : +- * Filter (67) + : +- * ColumnarToRow (66) + : +- Scan parquet default.web_site (65) + +- ReusedExchange (71) (1) Scan parquet default.store_sales @@ -160,7 +161,7 @@ Input [2]: [d_date_sk#25, d_date#26] (18) Filter [codegen id : 4] Input [2]: [d_date_sk#25, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 11192)) AND (d_date#26 <= 11206)) AND isnotnull(d_date_sk#25)) +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 2000-08-23)) AND (d_date#26 <= 2000-09-06)) AND isnotnull(d_date_sk#25)) (19) Project [codegen id : 4] Output [1]: [d_date_sk#25] @@ -318,157 +319,162 @@ Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#1 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -(53) Exchange +(53) Filter [codegen id : 14] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Condition : (isnotnull(wr_item_sk#98) AND isnotnull(wr_order_number#99)) + +(54) Exchange Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] -(54) Sort [codegen id : 15] +(55) Sort [codegen id : 15] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 -(55) Scan parquet default.web_sales +(56) Scan parquet default.web_sales Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 16] +(57) ColumnarToRow [codegen id : 16] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(57) Filter [codegen id : 16] +(58) Filter [codegen id : 16] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) -(58) Project [codegen id : 16] +(59) Project [codegen id : 16] Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(59) Exchange +(60) Exchange Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] -(60) Sort [codegen id : 17] +(61) Sort [codegen id : 17] Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 -(61) SortMergeJoin [codegen id : 18] +(62) SortMergeJoin [codegen id : 18] Left keys [2]: [wr_item_sk#98, wr_order_number#99] Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None -(62) Project [codegen id : 18] +(63) Project [codegen id : 18] Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -(63) Union +(64) Union -(64) Scan parquet default.web_site +(65) Scan parquet default.web_site Output [2]: [web_site_sk#113, web_site_id#114] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 19] +(66) ColumnarToRow [codegen id : 19] Input [2]: [web_site_sk#113, web_site_id#114] -(66) Filter [codegen id : 19] +(67) Filter [codegen id : 19] Input [2]: [web_site_sk#113, web_site_id#114] Condition : isnotnull(web_site_sk#113) -(67) BroadcastExchange +(68) BroadcastExchange Input [2]: [web_site_sk#113, web_site_id#114] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] -(68) BroadcastHashJoin [codegen id : 21] +(69) BroadcastHashJoin [codegen id : 21] Left keys [1]: [wsr_web_site_sk#92] Right keys [1]: [web_site_sk#113] Join condition: None -(69) Project [codegen id : 21] +(70) Project [codegen id : 21] Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] -(70) ReusedExchange [Reuses operator id: 20] +(71) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] -(71) BroadcastHashJoin [codegen id : 21] +(72) BroadcastHashJoin [codegen id : 21] Left keys [1]: [date_sk#93] Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None -(72) Project [codegen id : 21] +(73) Project [codegen id : 21] Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] -(73) HashAggregate [codegen id : 21] +(74) HashAggregate [codegen id : 21] Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Keys [1]: [web_site_id#114] Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -(74) Exchange +(75) Exchange Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] -(75) HashAggregate [codegen id : 22] +(76) HashAggregate [codegen id : 22] Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] Keys [1]: [web_site_id#114] Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#129, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS RETURNS#130, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#131, web channel AS channel#132, concat(web_site, web_site_id#114) AS id#133] -(76) Union +(77) Union -(77) Expand [codegen id : 23] +(78) Expand [codegen id : 23] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] -(78) HashAggregate [codegen id : 23] +(79) HashAggregate [codegen id : 23] Input [6]: [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] Keys [3]: [channel#134, id#135, spark_grouping_id#136] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] Results [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -(79) Exchange +(80) Exchange Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Arguments: hashpartitioning(channel#134, id#135, spark_grouping_id#136, 5), ENSURE_REQUIREMENTS, [id=#149] -(80) HashAggregate [codegen id : 24] +(81) HashAggregate [codegen id : 24] Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Keys [3]: [channel#134, id#135, spark_grouping_id#136] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] Aggregate Attributes [3]: [sum(sales#41)#150, sum(returns#42)#151, sum(profit#43)#152] Results [5]: [channel#134, id#135, sum(sales#41)#150 AS sales#153, sum(returns#42)#151 AS returns#154, sum(profit#43)#152 AS profit#155] -(81) TakeOrderedAndProject +(82) TakeOrderedAndProject Input [5]: [channel#134, id#135, sales#153, returns#154, profit#155] Arguments: 100, [channel#134 ASC NULLS FIRST, id#135 ASC NULLS FIRST], [channel#134, id#135, sales#153, returns#154, profit#155] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5 -ReusedExchange (82) +ReusedExchange (83) -(82) ReusedExchange [Reuses operator id: 20] +(83) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#49 IN dynamicpruning#50 -ReusedExchange (83) +ReusedExchange (84) -(83) ReusedExchange [Reuses operator id: 20] +(84) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt index 6239f80c2d62d..d3097eff9f8bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt @@ -114,10 +114,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Exchange [wr_item_sk,wr_order_number] #8 WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [wr_item_sk,wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter WholeStageCodegen (17) Sort [ws_item_sk,ws_order_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index f6fe8bee89369..81cded23e42db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (78) -+- * HashAggregate (77) - +- Exchange (76) - +- * HashAggregate (75) - +- * Expand (74) - +- Union (73) +TakeOrderedAndProject (79) ++- * HashAggregate (78) + +- Exchange (77) + +- * HashAggregate (76) + +- * Expand (75) + +- Union (74) :- * HashAggregate (25) : +- Exchange (24) : +- * HashAggregate (23) @@ -51,39 +51,41 @@ TakeOrderedAndProject (78) : +- * Filter (40) : +- * ColumnarToRow (39) : +- Scan parquet default.catalog_page (38) - +- * HashAggregate (72) - +- Exchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- Union (60) + +- * HashAggregate (73) + +- Exchange (72) + +- * HashAggregate (71) + +- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- Union (61) : : :- * Project (50) : : : +- * Filter (49) : : : +- * ColumnarToRow (48) : : : +- Scan parquet default.web_sales (47) - : : +- * Project (59) - : : +- * BroadcastHashJoin Inner BuildLeft (58) - : : :- BroadcastExchange (53) - : : : +- * ColumnarToRow (52) - : : : +- Scan parquet default.web_returns (51) - : : +- * Project (57) - : : +- * Filter (56) - : : +- * ColumnarToRow (55) - : : +- Scan parquet default.web_sales (54) - : +- ReusedExchange (61) - +- BroadcastExchange (67) - +- * Filter (66) - +- * ColumnarToRow (65) - +- Scan parquet default.web_site (64) + : : +- * Project (60) + : : +- * BroadcastHashJoin Inner BuildLeft (59) + : : :- BroadcastExchange (54) + : : : +- * Filter (53) + : : : +- * ColumnarToRow (52) + : : : +- Scan parquet default.web_returns (51) + : : +- * Project (58) + : : +- * Filter (57) + : : +- * ColumnarToRow (56) + : : +- Scan parquet default.web_sales (55) + : +- ReusedExchange (62) + +- BroadcastExchange (68) + +- * Filter (67) + +- * ColumnarToRow (66) + +- Scan parquet default.web_site (65) (1) Scan parquet default.store_sales Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ss_sold_date_sk#4 as bigint)), dynamicpruningexpression(cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), isnotnull(cast(ss_sold_date_sk#4 as bigint)), dynamicpruningexpression(cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -91,7 +93,7 @@ Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_s (3) Filter [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(cast(ss_store_sk#1 as bigint)) +Condition : (isnotnull(ss_store_sk#1) AND isnotnull(cast(ss_store_sk#1 as bigint))) (4) Project [codegen id : 1] Output [6]: [cast(ss_store_sk#1 as bigint) AS store_sk#6, cast(ss_sold_date_sk#4 as bigint) AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] @@ -130,7 +132,7 @@ Input [2]: [d_date_sk#22, d_date#23] (12) Filter [codegen id : 3] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11192)) AND (d_date#23 <= 11206)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) (13) Project [codegen id : 3] Output [1]: [d_date_sk#22] @@ -295,7 +297,7 @@ Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#8 Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#91), isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct @@ -315,145 +317,150 @@ Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#1 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -(53) BroadcastExchange +(53) Filter [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] +Condition : (isnotnull(wr_item_sk#98) AND isnotnull(wr_order_number#99)) -(54) Scan parquet default.web_sales +(54) BroadcastExchange +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false], input[1, bigint, false]),false), [id=#103] + +(55) Scan parquet default.web_sales Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(55) ColumnarToRow +(56) ColumnarToRow Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(56) Filter +(57) Filter Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) -(57) Project +(58) Project Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(58) BroadcastHashJoin [codegen id : 15] +(59) BroadcastHashJoin [codegen id : 15] Left keys [2]: [wr_item_sk#98, wr_order_number#99] Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None -(59) Project [codegen id : 15] +(60) Project [codegen id : 15] Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -(60) Union +(61) Union -(61) ReusedExchange [Reuses operator id: 14] +(62) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] -(62) BroadcastHashJoin [codegen id : 18] +(63) BroadcastHashJoin [codegen id : 18] Left keys [1]: [date_sk#93] Right keys [1]: [cast(d_date_sk#22 as bigint)] Join condition: None -(63) Project [codegen id : 18] +(64) Project [codegen id : 18] Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] -(64) Scan parquet default.web_site +(65) Scan parquet default.web_site Output [2]: [web_site_sk#112, web_site_id#113] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 17] +(66) ColumnarToRow [codegen id : 17] Input [2]: [web_site_sk#112, web_site_id#113] -(66) Filter [codegen id : 17] +(67) Filter [codegen id : 17] Input [2]: [web_site_sk#112, web_site_id#113] Condition : isnotnull(web_site_sk#112) -(67) BroadcastExchange +(68) BroadcastExchange Input [2]: [web_site_sk#112, web_site_id#113] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] -(68) BroadcastHashJoin [codegen id : 18] +(69) BroadcastHashJoin [codegen id : 18] Left keys [1]: [wsr_web_site_sk#92] Right keys [1]: [web_site_sk#112] Join condition: None -(69) Project [codegen id : 18] +(70) Project [codegen id : 18] Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] -(70) HashAggregate [codegen id : 18] +(71) HashAggregate [codegen id : 18] Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] Keys [1]: [web_site_id#113] Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -(71) Exchange +(72) Exchange Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] -(72) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] Keys [1]: [web_site_id#113] Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#128, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS RETURNS#129, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#130, web channel AS channel#131, concat(web_site, web_site_id#113) AS id#132] -(73) Union +(74) Union -(74) Expand [codegen id : 20] +(75) Expand [codegen id : 20] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] -(75) HashAggregate [codegen id : 20] +(76) HashAggregate [codegen id : 20] Input [6]: [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] Keys [3]: [channel#133, id#134, spark_grouping_id#135] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] Aggregate Attributes [6]: [sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Results [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -(76) Exchange +(77) Exchange Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] Arguments: hashpartitioning(channel#133, id#134, spark_grouping_id#135, 5), ENSURE_REQUIREMENTS, [id=#148] -(77) HashAggregate [codegen id : 21] +(78) HashAggregate [codegen id : 21] Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] Keys [3]: [channel#133, id#134, spark_grouping_id#135] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] Aggregate Attributes [3]: [sum(sales#41)#149, sum(returns#42)#150, sum(profit#43)#151] Results [5]: [channel#133, id#134, sum(sales#41)#149 AS sales#152, sum(returns#42)#150 AS returns#153, sum(profit#43)#151 AS profit#154] -(78) TakeOrderedAndProject +(79) TakeOrderedAndProject Input [5]: [channel#133, id#134, sales#152, returns#153, profit#154] Arguments: 100, [channel#133 ASC NULLS FIRST, id#134 ASC NULLS FIRST], [channel#133, id#134, sales#152, returns#153, profit#154] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5 -ReusedExchange (79) +ReusedExchange (80) -(79) ReusedExchange [Reuses operator id: 14] +(80) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#49 IN dynamicpruning#50 -ReusedExchange (80) +ReusedExchange (81) -(80) ReusedExchange [Reuses operator id: 14] +(81) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index 50dde50e729d1..3fd10cf9cb948 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -111,10 +111,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #8 WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [wr_item_sk,wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 Project [ws_item_sk,ws_web_site_sk,ws_order_number] Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 8e3bf5bf57bcf..322a882f92c48 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_item_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_item_id#14] -Condition : isnotnull(i_item_sk#13) +Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) (21) Scan parquet default.item Output [2]: [i_item_id#14, i_color#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] +PushedFilters: [In(i_color, [slate ,blanched ,burnished ]), IsNotNull(i_item_id)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_item_id#14, i_color#15] (23) Filter [codegen id : 3] Input [2]: [i_item_id#14, i_color#15] -Condition : i_color#15 IN (slate ,blanched ,burnished ) +Condition : (i_color#15 IN (slate ,blanched ,burnished ) AND isnotnull(i_item_id#14)) (24) Project [codegen id : 3] Output [1]: [i_item_id#14 AS i_item_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt index 97afd842e81eb..bfec5dcfdf5d0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_id,i_item_sk] + Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 8e3bf5bf57bcf..322a882f92c48 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_item_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_item_id#14] -Condition : isnotnull(i_item_sk#13) +Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) (21) Scan parquet default.item Output [2]: [i_item_id#14, i_color#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] +PushedFilters: [In(i_color, [slate ,blanched ,burnished ]), IsNotNull(i_item_id)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_item_id#14, i_color#15] (23) Filter [codegen id : 3] Input [2]: [i_item_id#14, i_color#15] -Condition : i_color#15 IN (slate ,blanched ,burnished ) +Condition : (i_color#15 IN (slate ,blanched ,burnished ) AND isnotnull(i_item_id#14)) (24) Project [codegen id : 3] Output [1]: [i_item_id#14 AS i_item_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 97afd842e81eb..bfec5dcfdf5d0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_id,i_item_sk] + Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index ee3abb39ed053..5c08ac23ebb55 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -84,7 +84,7 @@ TakeOrderedAndProject (79) Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct @@ -96,385 +96,406 @@ Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) (4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] +Output [2]: [d_date_sk#5, d_date#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#4, d_date#5] +Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 2] -Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Input [2]: [d_date_sk#5, d_date#6] +Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#6)) (7) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#6, d_week_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#6, d_week_seq#7] (9) Filter [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = Subquery scalar-subquery#7, [id=#8])) +Input [2]: [d_date#6, d_week_seq#7] +Condition : ((isnotnull(d_week_seq#7) AND (d_week_seq#7 = Subquery scalar-subquery#8, [id=#9])) AND isnotnull(d_date#6)) (10) Project [codegen id : 1] -Output [1]: [d_date#5 AS d_date#5#9] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#6 AS d_date#6#10] +Input [2]: [d_date#6, d_week_seq#7] (11) BroadcastExchange -Input [1]: [d_date#5#9] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] +Input [1]: [d_date#6#10] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#11] (12) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#9] +Left keys [1]: [d_date#6] +Right keys [1]: [d_date#6#10] Join condition: None (13) Project [codegen id : 2] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +Output [1]: [d_date_sk#5] +Input [2]: [d_date_sk#5, d_date#6] (14) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (15) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#4] +Right keys [1]: [d_date_sk#5] Join condition: None (16) Project [codegen id : 4] Output [2]: [ss_item_sk#1, ss_ext_sales_price#2] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#4] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5] (17) Scan parquet default.item -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#12, i_item_id#13] +Input [2]: [i_item_sk#13, i_item_id#14] (19) Filter [codegen id : 3] -Input [2]: [i_item_sk#12, i_item_id#13] -Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) +Input [2]: [i_item_sk#13, i_item_id#14] +Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) (20) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] +Input [2]: [i_item_sk#13, i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] (21) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#12] +Right keys [1]: [i_item_sk#13] Join condition: None (22) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#2, i_item_id#13] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, i_item_sk#12, i_item_id#13] +Output [2]: [ss_ext_sales_price#2, i_item_id#14] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, i_item_sk#13, i_item_id#14] (23) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#2, i_item_id#13] -Keys [1]: [i_item_id#13] +Input [2]: [ss_ext_sales_price#2, i_item_id#14] +Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#13, sum#16] +Aggregate Attributes [1]: [sum#16] +Results [2]: [i_item_id#14, sum#17] (24) Exchange -Input [2]: [i_item_id#13, sum#16] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [2]: [i_item_id#14, sum#17] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#18] (25) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#13, sum#16] -Keys [1]: [i_item_id#13] +Input [2]: [i_item_id#14, sum#17] +Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#18] -Results [2]: [i_item_id#13 AS item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#18,17,2) AS ss_item_rev#20] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#19] +Results [2]: [i_item_id#14 AS item_id#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS ss_item_rev#21] (26) Filter [codegen id : 15] -Input [2]: [item_id#19, ss_item_rev#20] -Condition : isnotnull(ss_item_rev#20) +Input [2]: [item_id#20, ss_item_rev#21] +Condition : isnotnull(ss_item_rev#21) (27) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +Output [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cs_sold_date_sk#24), dynamicpruningexpression(cs_sold_date_sk#24 IN dynamicpruning#25)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +Input [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] (29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#21) +Input [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +Condition : isnotnull(cs_item_sk#22) (30) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] +Output [2]: [d_date_sk#5, d_date#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (31) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#4, d_date#5] +Input [2]: [d_date_sk#5, d_date#6] (32) Filter [codegen id : 6] -Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Input [2]: [d_date_sk#5, d_date#6] +Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#6)) (33) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#6, d_week_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (34) ColumnarToRow [codegen id : 5] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#6, d_week_seq#7] (35) Filter [codegen id : 5] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) +Input [2]: [d_date#6, d_week_seq#7] +Condition : ((isnotnull(d_week_seq#7) AND (d_week_seq#7 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) AND isnotnull(d_date#6)) (36) Project [codegen id : 5] -Output [1]: [d_date#5 AS d_date#5#24] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#6 AS d_date#6#26] +Input [2]: [d_date#6, d_week_seq#7] (37) BroadcastExchange -Input [1]: [d_date#5#24] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] +Input [1]: [d_date#6#26] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#27] (38) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#24] +Left keys [1]: [d_date#6] +Right keys [1]: [d_date#6#26] Join condition: None (39) Project [codegen id : 6] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +Output [1]: [d_date_sk#5] +Input [2]: [d_date_sk#5, d_date#6] (40) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#4] +Left keys [1]: [cs_sold_date_sk#24] +Right keys [1]: [d_date_sk#5] Join condition: None (42) Project [codegen id : 8] -Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#4] +Output [2]: [cs_item_sk#22, cs_ext_sales_price#23] +Input [4]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24, d_date_sk#5] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#13, i_item_id#14] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#13] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#13] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#12, i_item_id#13] +Output [2]: [cs_ext_sales_price#23, i_item_id#14] +Input [4]: [cs_item_sk#22, cs_ext_sales_price#23, i_item_sk#13, i_item_id#14] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#13, sum#28] +Input [2]: [cs_ext_sales_price#23, i_item_id#14] +Keys [1]: [i_item_id#14] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#23))] +Aggregate Attributes [1]: [sum#29] +Results [2]: [i_item_id#14, sum#30] (47) Exchange -Input [2]: [i_item_id#13, sum#28] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [i_item_id#14, sum#30] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#31] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#13, sum#28] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#13 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] +Input [2]: [i_item_id#14, sum#30] +Keys [1]: [i_item_id#14] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#23))#32] +Results [2]: [i_item_id#14 AS item_id#33, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#23))#32,17,2) AS cs_item_rev#34] (49) Filter [codegen id : 9] -Input [2]: [item_id#31, cs_item_rev#32] -Condition : isnotnull(cs_item_rev#32) +Input [2]: [item_id#33, cs_item_rev#34] +Condition : isnotnull(cs_item_rev#34) (50) BroadcastExchange -Input [2]: [item_id#31, cs_item_rev#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] +Input [2]: [item_id#33, cs_item_rev#34] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#35] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#33] +Join condition: ((((cast(ss_item_rev#21 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#34)), DecimalType(19,3), true)) AND (cast(ss_item_rev#21 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#34)), DecimalType(20,3), true))) AND (cast(cs_item_rev#34 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#21)), DecimalType(19,3), true))) AND (cast(cs_item_rev#34 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#21)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] -Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] +Output [3]: [item_id#20, ss_item_rev#21, cs_item_rev#34] +Input [4]: [item_id#20, ss_item_rev#21, item_id#33, cs_item_rev#34] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#34) +Input [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#36) (56) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] +Output [2]: [d_date_sk#5, d_date#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (57) ColumnarToRow [codegen id : 11] -Input [2]: [d_date_sk#4, d_date#5] +Input [2]: [d_date_sk#5, d_date#6] (58) Filter [codegen id : 11] -Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Input [2]: [d_date_sk#5, d_date#6] +Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#6)) (59) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#6, d_week_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (60) ColumnarToRow [codegen id : 10] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#6, d_week_seq#7] (61) Filter [codegen id : 10] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) +Input [2]: [d_date#6, d_week_seq#7] +Condition : ((isnotnull(d_week_seq#7) AND (d_week_seq#7 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) AND isnotnull(d_date#6)) (62) Project [codegen id : 10] -Output [1]: [d_date#5 AS d_date#5#37] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#6 AS d_date#6#40] +Input [2]: [d_date#6, d_week_seq#7] (63) BroadcastExchange -Input [1]: [d_date#5#37] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] +Input [1]: [d_date#6#40] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#41] (64) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#37] +Left keys [1]: [d_date#6] +Right keys [1]: [d_date#6#40] Join condition: None (65) Project [codegen id : 11] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +Output [1]: [d_date_sk#5] +Input [2]: [d_date_sk#5, d_date#6] (66) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (67) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#4] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#5] Join condition: None (68) Project [codegen id : 13] -Output [2]: [ws_item_sk#34, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#4] +Output [2]: [ws_item_sk#36, ws_ext_sales_price#37] +Input [4]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38, d_date_sk#5] (69) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#13, i_item_id#14] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [ws_item_sk#36] +Right keys [1]: [i_item_sk#13] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#35, i_item_id#13] -Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, i_item_sk#12, i_item_id#13] +Output [2]: [ws_ext_sales_price#37, i_item_id#14] +Input [4]: [ws_item_sk#36, ws_ext_sales_price#37, i_item_sk#13, i_item_id#14] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#35, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#13, sum#41] +Input [2]: [ws_ext_sales_price#37, i_item_id#14] +Keys [1]: [i_item_id#14] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#37))] +Aggregate Attributes [1]: [sum#43] +Results [2]: [i_item_id#14, sum#44] (73) Exchange -Input [2]: [i_item_id#13, sum#41] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [2]: [i_item_id#14, sum#44] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#45] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#13, sum#41] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#13 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] +Input [2]: [i_item_id#14, sum#44] +Keys [1]: [i_item_id#14] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#37))#46] +Results [2]: [i_item_id#14 AS item_id#47, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#37))#46,17,2) AS ws_item_rev#48] (75) Filter [codegen id : 14] -Input [2]: [item_id#44, ws_item_rev#45] -Condition : isnotnull(ws_item_rev#45) +Input [2]: [item_id#47, ws_item_rev#48] +Condition : isnotnull(ws_item_rev#48) (76) BroadcastExchange -Input [2]: [item_id#44, ws_item_rev#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] +Input [2]: [item_id#47, ws_item_rev#48] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#49] (77) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#47] +Join condition: ((((((((cast(ss_item_rev#21 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#48)), DecimalType(19,3), true)) AND (cast(ss_item_rev#21 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#48)), DecimalType(20,3), true))) AND (cast(cs_item_rev#34 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#48)), DecimalType(19,3), true))) AND (cast(cs_item_rev#34 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#48)), DecimalType(20,3), true))) AND (cast(ws_item_rev#48 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#21)), DecimalType(19,3), true))) AND (cast(ws_item_rev#48 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#21)), DecimalType(20,3), true))) AND (cast(ws_item_rev#48 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#34)), DecimalType(19,3), true))) AND (cast(ws_item_rev#48 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#34)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] +Output [8]: [item_id#20, ss_item_rev#21, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#50, cs_item_rev#34, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#34 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#51, ws_item_rev#48, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#48 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#52, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#53] +Input [5]: [item_id#20, ss_item_rev#21, cs_item_rev#34, item_id#47, ws_item_rev#48] (79) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Input [8]: [item_id#20, ss_item_rev#21, ss_dev#50, cs_item_rev#34, cs_dev#51, ws_item_rev#48, ws_dev#52, average#53] +Arguments: 100, [item_id#20 ASC NULLS FIRST, ss_item_rev#21 ASC NULLS FIRST], [item_id#20, ss_item_rev#21, ss_dev#50, cs_item_rev#34, cs_dev#51, ws_item_rev#48, ws_dev#52, average#53] ===== Subqueries ===== -Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#7, [id=#8] -* Project (83) -+- * Filter (82) - +- * ColumnarToRow (81) - +- Scan parquet default.date_dim (80) +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +ReusedExchange (80) -(80) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +(80) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#5] + +Subquery:2 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#8, [id=#9] +* Project (84) ++- * Filter (83) + +- * ColumnarToRow (82) + +- Scan parquet default.date_dim (81) + + +(81) Scan parquet default.date_dim +Output [2]: [d_date#6, d_week_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(81) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] +(82) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#6, d_week_seq#7] + +(83) Filter [codegen id : 1] +Input [2]: [d_date#6, d_week_seq#7] +Condition : (isnotnull(d_date#6) AND (d_date#6 = 2000-01-03)) + +(84) Project [codegen id : 1] +Output [1]: [d_week_seq#7] +Input [2]: [d_date#6, d_week_seq#7] + +Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#24 IN dynamicpruning#25 +ReusedExchange (85) + + +(85) ReusedExchange [Reuses operator id: unknown] +Output [1]: [d_date_sk#5] + +Subquery:4 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] -(82) Filter [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_date#5) AND (d_date#5 = 10959)) +Subquery:5 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#39 +ReusedExchange (86) -(83) Project [codegen id : 1] -Output [1]: [d_week_seq#6] -Input [2]: [d_date#5, d_week_seq#6] -Subquery:2 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] +(86) ReusedExchange [Reuses operator id: unknown] +Output [1]: [d_date_sk#5] -Subquery:3 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] +Subquery:6 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt index 08a45c563fcc0..67f864915caca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt @@ -18,12 +18,14 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #2 WholeStageCodegen (2) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date,d_date_sk] + Filter [d_date_sk,d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] @@ -31,8 +33,8 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev BroadcastExchange #3 WholeStageCodegen (1) Project [d_date] - Filter [d_date,d_week_seq] - Subquery #1 + Filter [d_week_seq,d_date] + Subquery #2 WholeStageCodegen (1) Project [d_week_seq] Filter [d_date] @@ -66,33 +68,35 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + ReusedExchange [d_date_sk] #7 InputAdapter - BroadcastExchange #7 + BroadcastExchange #8 WholeStageCodegen (6) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date,d_date_sk] + Filter [d_date_sk,d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #8 + BroadcastExchange #9 WholeStageCodegen (5) Project [d_date] - Filter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #1 + Filter [d_week_seq,d_date] + ReusedSubquery [d_week_seq] #2 ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter ReusedExchange [i_item_sk,i_item_id] #4 InputAdapter - BroadcastExchange #9 + BroadcastExchange #10 WholeStageCodegen (14) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] InputAdapter - Exchange [i_item_id] #10 + Exchange [i_item_id] #11 WholeStageCodegen (13) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -103,21 +107,23 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + ReusedExchange [d_date_sk] #12 InputAdapter - BroadcastExchange #11 + BroadcastExchange #13 WholeStageCodegen (11) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date,d_date_sk] + Filter [d_date_sk,d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #12 + BroadcastExchange #14 WholeStageCodegen (10) Project [d_date] - Filter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #1 + Filter [d_week_seq,d_date] + ReusedSubquery [d_week_seq] #2 ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index dd37b8801e957..499559c701c71 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -84,7 +84,7 @@ TakeOrderedAndProject (79) Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct @@ -96,385 +96,406 @@ Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) (4) Scan parquet default.item -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#5, i_item_id#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#4, i_item_id#5] +Input [2]: [i_item_sk#5, i_item_id#6] (6) Filter [codegen id : 1] -Input [2]: [i_item_sk#4, i_item_id#5] -Condition : (isnotnull(i_item_sk#4) AND isnotnull(i_item_id#5)) +Input [2]: [i_item_sk#5, i_item_id#6] +Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) (7) BroadcastExchange -Input [2]: [i_item_sk#4, i_item_id#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#6] +Input [2]: [i_item_sk#5, i_item_id#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] (8) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#4] +Right keys [1]: [i_item_sk#5] Join condition: None (9) Project [codegen id : 4] -Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#5] -Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#4, i_item_id#5] +Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6] +Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6] (10) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] +Output [2]: [d_date_sk#8, d_date#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (11) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#7, d_date#8] +Input [2]: [d_date_sk#8, d_date#9] (12) Filter [codegen id : 3] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Input [2]: [d_date_sk#8, d_date#9] +Condition : (isnotnull(d_date_sk#8) AND isnotnull(d_date#9)) (13) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#9, d_week_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (14) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#9, d_week_seq#10] (15) Filter [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = Subquery scalar-subquery#10, [id=#11])) +Input [2]: [d_date#9, d_week_seq#10] +Condition : ((isnotnull(d_week_seq#10) AND (d_week_seq#10 = Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date#9)) (16) Project [codegen id : 2] -Output [1]: [d_date#8 AS d_date#8#12] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#9 AS d_date#9#13] +Input [2]: [d_date#9, d_week_seq#10] (17) BroadcastExchange -Input [1]: [d_date#8#12] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] +Input [1]: [d_date#9#13] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#14] (18) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#12] +Left keys [1]: [d_date#9] +Right keys [1]: [d_date#9#13] Join condition: None (19) Project [codegen id : 3] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] +Output [1]: [d_date_sk#8] +Input [2]: [d_date_sk#8, d_date#9] (20) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [d_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (21) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#8] Join condition: None (22) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#2, i_item_id#5] -Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#5, d_date_sk#7] +Output [2]: [ss_ext_sales_price#2, i_item_id#6] +Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, d_date_sk#8] (23) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#2, i_item_id#5] -Keys [1]: [i_item_id#5] +Input [2]: [ss_ext_sales_price#2, i_item_id#6] +Keys [1]: [i_item_id#6] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#5, sum#16] +Aggregate Attributes [1]: [sum#16] +Results [2]: [i_item_id#6, sum#17] (24) Exchange -Input [2]: [i_item_id#5, sum#16] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [2]: [i_item_id#6, sum#17] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#18] (25) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#5, sum#16] -Keys [1]: [i_item_id#5] +Input [2]: [i_item_id#6, sum#17] +Keys [1]: [i_item_id#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#18] -Results [2]: [i_item_id#5 AS item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#18,17,2) AS ss_item_rev#20] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#19] +Results [2]: [i_item_id#6 AS item_id#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS ss_item_rev#21] (26) Filter [codegen id : 15] -Input [2]: [item_id#19, ss_item_rev#20] -Condition : isnotnull(ss_item_rev#20) +Input [2]: [item_id#20, ss_item_rev#21] +Condition : isnotnull(ss_item_rev#21) (27) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +Output [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cs_sold_date_sk#24), dynamicpruningexpression(cs_sold_date_sk#24 IN dynamicpruning#25)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +Input [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] (29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#21) +Input [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +Condition : isnotnull(cs_item_sk#22) (30) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#5, i_item_id#6] (31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#4] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#5] Join condition: None (32) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5] -Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#4, i_item_id#5] +Output [3]: [cs_ext_sales_price#23, cs_sold_date_sk#24, i_item_id#6] +Input [5]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24, i_item_sk#5, i_item_id#6] (33) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] +Output [2]: [d_date_sk#8, d_date#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (34) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#7, d_date#8] +Input [2]: [d_date_sk#8, d_date#9] (35) Filter [codegen id : 7] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Input [2]: [d_date_sk#8, d_date#9] +Condition : (isnotnull(d_date_sk#8) AND isnotnull(d_date#9)) (36) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#9, d_week_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#9, d_week_seq#10] (38) Filter [codegen id : 6] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) +Input [2]: [d_date#9, d_week_seq#10] +Condition : ((isnotnull(d_week_seq#10) AND (d_week_seq#10 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date#9)) (39) Project [codegen id : 6] -Output [1]: [d_date#8 AS d_date#8#24] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#9 AS d_date#9#26] +Input [2]: [d_date#9, d_week_seq#10] (40) BroadcastExchange -Input [1]: [d_date#8#24] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] +Input [1]: [d_date#9#26] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#27] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#24] +Left keys [1]: [d_date#9] +Right keys [1]: [d_date#9#26] Join condition: None (42) Project [codegen id : 7] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] +Output [1]: [d_date_sk#8] +Input [2]: [d_date_sk#8, d_date#9] (43) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [cs_sold_date_sk#24] +Right keys [1]: [d_date_sk#8] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#5] -Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5, d_date_sk#7] +Output [2]: [cs_ext_sales_price#23, i_item_id#6] +Input [4]: [cs_ext_sales_price#23, cs_sold_date_sk#24, i_item_id#6, d_date_sk#8] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#5, sum#28] +Input [2]: [cs_ext_sales_price#23, i_item_id#6] +Keys [1]: [i_item_id#6] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#23))] +Aggregate Attributes [1]: [sum#29] +Results [2]: [i_item_id#6, sum#30] (47) Exchange -Input [2]: [i_item_id#5, sum#28] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [i_item_id#6, sum#30] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#31] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#5, sum#28] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#5 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] +Input [2]: [i_item_id#6, sum#30] +Keys [1]: [i_item_id#6] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#23))#32] +Results [2]: [i_item_id#6 AS item_id#33, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#23))#32,17,2) AS cs_item_rev#34] (49) Filter [codegen id : 9] -Input [2]: [item_id#31, cs_item_rev#32] -Condition : isnotnull(cs_item_rev#32) +Input [2]: [item_id#33, cs_item_rev#34] +Condition : isnotnull(cs_item_rev#34) (50) BroadcastExchange -Input [2]: [item_id#31, cs_item_rev#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] +Input [2]: [item_id#33, cs_item_rev#34] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#35] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#33] +Join condition: ((((cast(ss_item_rev#21 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#34)), DecimalType(19,3), true)) AND (cast(ss_item_rev#21 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#34)), DecimalType(20,3), true))) AND (cast(cs_item_rev#34 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#21)), DecimalType(19,3), true))) AND (cast(cs_item_rev#34 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#21)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] -Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] +Output [3]: [item_id#20, ss_item_rev#21, cs_item_rev#34] +Input [4]: [item_id#20, ss_item_rev#21, item_id#33, cs_item_rev#34] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#34) +Input [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#36) (56) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#5, i_item_id#6] (57) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#4] +Left keys [1]: [ws_item_sk#36] +Right keys [1]: [i_item_sk#5] Join condition: None (58) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5] -Input [5]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_sk#4, i_item_id#5] +Output [3]: [ws_ext_sales_price#37, ws_sold_date_sk#38, i_item_id#6] +Input [5]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38, i_item_sk#5, i_item_id#6] (59) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] +Output [2]: [d_date_sk#8, d_date#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (60) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#7, d_date#8] +Input [2]: [d_date_sk#8, d_date#9] (61) Filter [codegen id : 12] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Input [2]: [d_date_sk#8, d_date#9] +Condition : (isnotnull(d_date_sk#8) AND isnotnull(d_date#9)) (62) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#9, d_week_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (63) ColumnarToRow [codegen id : 11] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#9, d_week_seq#10] (64) Filter [codegen id : 11] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) +Input [2]: [d_date#9, d_week_seq#10] +Condition : ((isnotnull(d_week_seq#10) AND (d_week_seq#10 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date#9)) (65) Project [codegen id : 11] -Output [1]: [d_date#8 AS d_date#8#37] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#9 AS d_date#9#40] +Input [2]: [d_date#9, d_week_seq#10] (66) BroadcastExchange -Input [1]: [d_date#8#37] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] +Input [1]: [d_date#9#40] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#41] (67) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#37] +Left keys [1]: [d_date#9] +Right keys [1]: [d_date#9#40] Join condition: None (68) Project [codegen id : 12] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] +Output [1]: [d_date_sk#8] +Input [2]: [d_date_sk#8, d_date#9] (69) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [1]: [d_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#8] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#35, i_item_id#5] -Input [4]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5, d_date_sk#7] +Output [2]: [ws_ext_sales_price#37, i_item_id#6] +Input [4]: [ws_ext_sales_price#37, ws_sold_date_sk#38, i_item_id#6, d_date_sk#8] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#35, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#5, sum#41] +Input [2]: [ws_ext_sales_price#37, i_item_id#6] +Keys [1]: [i_item_id#6] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#37))] +Aggregate Attributes [1]: [sum#43] +Results [2]: [i_item_id#6, sum#44] (73) Exchange -Input [2]: [i_item_id#5, sum#41] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [2]: [i_item_id#6, sum#44] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#45] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#5, sum#41] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#5 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] +Input [2]: [i_item_id#6, sum#44] +Keys [1]: [i_item_id#6] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#37))#46] +Results [2]: [i_item_id#6 AS item_id#47, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#37))#46,17,2) AS ws_item_rev#48] (75) Filter [codegen id : 14] -Input [2]: [item_id#44, ws_item_rev#45] -Condition : isnotnull(ws_item_rev#45) +Input [2]: [item_id#47, ws_item_rev#48] +Condition : isnotnull(ws_item_rev#48) (76) BroadcastExchange -Input [2]: [item_id#44, ws_item_rev#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] +Input [2]: [item_id#47, ws_item_rev#48] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#49] (77) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#47] +Join condition: ((((((((cast(ss_item_rev#21 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#48)), DecimalType(19,3), true)) AND (cast(ss_item_rev#21 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#48)), DecimalType(20,3), true))) AND (cast(cs_item_rev#34 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#48)), DecimalType(19,3), true))) AND (cast(cs_item_rev#34 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#48)), DecimalType(20,3), true))) AND (cast(ws_item_rev#48 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#21)), DecimalType(19,3), true))) AND (cast(ws_item_rev#48 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#21)), DecimalType(20,3), true))) AND (cast(ws_item_rev#48 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#34)), DecimalType(19,3), true))) AND (cast(ws_item_rev#48 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#34)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] +Output [8]: [item_id#20, ss_item_rev#21, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#50, cs_item_rev#34, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#34 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#51, ws_item_rev#48, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#48 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#52, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#53] +Input [5]: [item_id#20, ss_item_rev#21, cs_item_rev#34, item_id#47, ws_item_rev#48] (79) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Input [8]: [item_id#20, ss_item_rev#21, ss_dev#50, cs_item_rev#34, cs_dev#51, ws_item_rev#48, ws_dev#52, average#53] +Arguments: 100, [item_id#20 ASC NULLS FIRST, ss_item_rev#21 ASC NULLS FIRST], [item_id#20, ss_item_rev#21, ss_dev#50, cs_item_rev#34, cs_dev#51, ws_item_rev#48, ws_dev#52, average#53] ===== Subqueries ===== -Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#10, [id=#11] -* Project (83) -+- * Filter (82) - +- * ColumnarToRow (81) - +- Scan parquet default.date_dim (80) +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +ReusedExchange (80) -(80) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +(80) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#8] + +Subquery:2 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* Project (84) ++- * Filter (83) + +- * ColumnarToRow (82) + +- Scan parquet default.date_dim (81) + + +(81) Scan parquet default.date_dim +Output [2]: [d_date#9, d_week_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(81) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] +(82) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#9, d_week_seq#10] + +(83) Filter [codegen id : 1] +Input [2]: [d_date#9, d_week_seq#10] +Condition : (isnotnull(d_date#9) AND (d_date#9 = 2000-01-03)) + +(84) Project [codegen id : 1] +Output [1]: [d_week_seq#10] +Input [2]: [d_date#9, d_week_seq#10] + +Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#24 IN dynamicpruning#25 +ReusedExchange (85) + + +(85) ReusedExchange [Reuses operator id: unknown] +Output [1]: [d_date_sk#8] + +Subquery:4 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -(82) Filter [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_date#8) AND (d_date#8 = 10959)) +Subquery:5 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#39 +ReusedExchange (86) -(83) Project [codegen id : 1] -Output [1]: [d_week_seq#9] -Input [2]: [d_date#8, d_week_seq#9] -Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] +(86) ReusedExchange [Reuses operator id: unknown] +Output [1]: [d_date_sk#8] -Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] +Subquery:6 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 6ef27d625b970..ea72681c2f5f1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -18,19 +18,21 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #2 + BroadcastExchange #3 WholeStageCodegen (1) Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #3 + BroadcastExchange #2 WholeStageCodegen (3) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date,d_date_sk] + Filter [d_date_sk,d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] @@ -38,8 +40,8 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev BroadcastExchange #4 WholeStageCodegen (2) Project [d_date] - Filter [d_date,d_week_seq] - Subquery #1 + Filter [d_week_seq,d_date] + Subquery #2 WholeStageCodegen (1) Project [d_week_seq] Filter [d_date] @@ -66,33 +68,35 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + ReusedExchange [d_date_sk] #7 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 + ReusedExchange [i_item_sk,i_item_id] #3 InputAdapter - BroadcastExchange #7 + BroadcastExchange #8 WholeStageCodegen (7) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date,d_date_sk] + Filter [d_date_sk,d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #8 + BroadcastExchange #9 WholeStageCodegen (6) Project [d_date] - Filter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #1 + Filter [d_week_seq,d_date] + ReusedSubquery [d_week_seq] #2 ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter - BroadcastExchange #9 + BroadcastExchange #10 WholeStageCodegen (14) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] InputAdapter - Exchange [i_item_id] #10 + Exchange [i_item_id] #11 WholeStageCodegen (13) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -103,23 +107,25 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + ReusedExchange [d_date_sk] #12 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 + ReusedExchange [i_item_sk,i_item_id] #3 InputAdapter - BroadcastExchange #11 + BroadcastExchange #13 WholeStageCodegen (12) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date,d_date_sk] + Filter [d_date_sk,d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #12 + BroadcastExchange #14 WholeStageCodegen (11) Project [d_date] - Filter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #1 + Filter [d_week_seq,d_date] + ReusedSubquery [d_week_seq] #2 ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 10b8d0d9e7f05..0085627d7f485 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_item_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_item_id#14] -Condition : isnotnull(i_item_sk#13) +Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) (21) Scan parquet default.item Output [2]: [i_item_id#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music ), IsNotNull(i_item_id)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_item_id#14, i_category#15] (23) Filter [codegen id : 3] Input [2]: [i_item_id#14, i_category#15] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) +Condition : ((isnotnull(i_category#15) AND (i_category#15 = Music )) AND isnotnull(i_item_id#14)) (24) Project [codegen id : 3] Output [1]: [i_item_id#14 AS i_item_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt index 1a882da5cd0d9..45041ad094f7b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_id,i_item_sk] + Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 10b8d0d9e7f05..0085627d7f485 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_item_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_item_id#14] -Condition : isnotnull(i_item_sk#13) +Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) (21) Scan parquet default.item Output [2]: [i_item_id#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music ), IsNotNull(i_item_id)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_item_id#14, i_category#15] (23) Filter [codegen id : 3] Input [2]: [i_item_id#14, i_category#15] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) +Condition : ((isnotnull(i_category#15) AND (i_category#15 = Music )) AND isnotnull(i_item_id#14)) (24) Project [codegen id : 3] Output [1]: [i_item_id#14 AS i_item_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 1a882da5cd0d9..45041ad094f7b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_id,i_item_sk] + Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt index 97d7a81baf2a7..58598de9e10c5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt @@ -1,62 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (51) -+- * HashAggregate (50) - +- Exchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin Inner BuildLeft (46) - :- BroadcastExchange (42) - : +- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (34) - : : +- SortMergeJoin LeftAnti (33) - : : :- SortMergeJoin LeftAnti (25) - : : : :- SortMergeJoin LeftSemi (17) +TakeOrderedAndProject (54) ++- * HashAggregate (53) + +- Exchange (52) + +- * HashAggregate (51) + +- * Project (50) + +- * BroadcastHashJoin Inner BuildLeft (49) + :- BroadcastExchange (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (37) + : : +- SortMergeJoin LeftAnti (36) + : : :- SortMergeJoin LeftAnti (27) + : : : :- SortMergeJoin LeftSemi (18) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- * Sort (16) - : : : : +- Exchange (15) - : : : : +- * Project (14) - : : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : : :- * ColumnarToRow (7) - : : : : : +- Scan parquet default.store_sales (6) - : : : : +- BroadcastExchange (12) - : : : : +- * Project (11) - : : : : +- * Filter (10) - : : : : +- * ColumnarToRow (9) - : : : : +- Scan parquet default.date_dim (8) - : : : +- * Sort (24) - : : : +- Exchange (23) - : : : +- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * ColumnarToRow (19) - : : : : +- Scan parquet default.web_sales (18) - : : : +- ReusedExchange (20) - : : +- * Sort (32) - : : +- Exchange (31) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * ColumnarToRow (27) - : : : +- Scan parquet default.catalog_sales (26) - : : +- ReusedExchange (28) - : +- BroadcastExchange (39) - : +- * Project (38) - : +- * Filter (37) - : +- * ColumnarToRow (36) - : +- Scan parquet default.customer_address (35) - +- * Filter (45) - +- * ColumnarToRow (44) - +- Scan parquet default.customer_demographics (43) + : : : : +- * Sort (17) + : : : : +- Exchange (16) + : : : : +- * Project (15) + : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * Filter (11) + : : : : +- * ColumnarToRow (10) + : : : : +- Scan parquet default.date_dim (9) + : : : +- * Sort (26) + : : : +- Exchange (25) + : : : +- * Project (24) + : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : :- * Filter (21) + : : : : +- * ColumnarToRow (20) + : : : : +- Scan parquet default.web_sales (19) + : : : +- ReusedExchange (22) + : : +- * Sort (35) + : : +- Exchange (34) + : : +- * Project (33) + : : +- * BroadcastHashJoin Inner BuildRight (32) + : : :- * Filter (30) + : : : +- * ColumnarToRow (29) + : : : +- Scan parquet default.catalog_sales (28) + : : +- ReusedExchange (31) + : +- BroadcastExchange (42) + : +- * Project (41) + : +- * Filter (40) + : +- * ColumnarToRow (39) + : +- Scan parquet default.customer_address (38) + +- * Filter (48) + +- * ColumnarToRow (47) + +- Scan parquet default.customer_demographics (46) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -64,7 +67,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -79,220 +82,235 @@ Output [2]: [ss_customer_sk#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -(8) Scan parquet default.date_dim +(8) Filter [codegen id : 4] +Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] +Condition : isnotnull(ss_customer_sk#5) + +(9) Scan parquet default.date_dim Output [3]: [d_date_sk#8, d_year#9, d_moy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct -(9) ColumnarToRow [codegen id : 3] +(10) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -(10) Filter [codegen id : 3] +(11) Filter [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] Condition : (((((isnotnull(d_year#9) AND isnotnull(d_moy#10)) AND (d_year#9 = 2001)) AND (d_moy#10 >= 4)) AND (d_moy#10 <= 6)) AND isnotnull(d_date_sk#8)) -(11) Project [codegen id : 3] +(12) Project [codegen id : 3] Output [1]: [d_date_sk#8] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -(12) BroadcastExchange +(13) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(13) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#8] Join condition: None -(14) Project [codegen id : 4] +(15) Project [codegen id : 4] Output [1]: [ss_customer_sk#5] Input [3]: [ss_customer_sk#5, ss_sold_date_sk#6, d_date_sk#8] -(15) Exchange +(16) Exchange Input [1]: [ss_customer_sk#5] Arguments: hashpartitioning(ss_customer_sk#5, 5), ENSURE_REQUIREMENTS, [id=#12] -(16) Sort [codegen id : 5] +(17) Sort [codegen id : 5] Input [1]: [ss_customer_sk#5] Arguments: [ss_customer_sk#5 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin +(18) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#5] Join condition: None -(18) Scan parquet default.web_sales +(19) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#14), dynamicpruningexpression(ws_sold_date_sk#14 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(20) ColumnarToRow [codegen id : 7] +Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] + +(21) Filter [codegen id : 7] Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] +Condition : isnotnull(ws_bill_customer_sk#13) -(20) ReusedExchange [Reuses operator id: 12] +(22) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -(21) BroadcastHashJoin [codegen id : 7] +(23) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] Right keys [1]: [d_date_sk#8] Join condition: None -(22) Project [codegen id : 7] +(24) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13] Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] -(23) Exchange +(25) Exchange Input [1]: [ws_bill_customer_sk#13] Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#15] -(24) Sort [codegen id : 8] +(26) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#13] Arguments: [ws_bill_customer_sk#13 ASC NULLS FIRST], false, 0 -(25) SortMergeJoin +(27) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#13] Join condition: None -(26) Scan parquet default.catalog_sales +(28) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 10] +(29) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -(28) ReusedExchange [Reuses operator id: 12] +(30) Filter [codegen id : 10] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Condition : isnotnull(cs_ship_customer_sk#16) + +(31) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -(29) BroadcastHashJoin [codegen id : 10] +(32) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#8] Join condition: None -(30) Project [codegen id : 10] +(33) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#16] Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] -(31) Exchange +(34) Exchange Input [1]: [cs_ship_customer_sk#16] Arguments: hashpartitioning(cs_ship_customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] -(32) Sort [codegen id : 11] +(35) Sort [codegen id : 11] Input [1]: [cs_ship_customer_sk#16] Arguments: [cs_ship_customer_sk#16 ASC NULLS FIRST], false, 0 -(33) SortMergeJoin +(36) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#16] Join condition: None -(34) Project [codegen id : 13] +(37) Project [codegen id : 13] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(35) Scan parquet default.customer_address +(38) Scan parquet default.customer_address Output [2]: [ca_address_sk#19, ca_state#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 12] +(39) ColumnarToRow [codegen id : 12] Input [2]: [ca_address_sk#19, ca_state#20] -(37) Filter [codegen id : 12] +(40) Filter [codegen id : 12] Input [2]: [ca_address_sk#19, ca_state#20] Condition : (ca_state#20 IN (KY,GA,NM) AND isnotnull(ca_address_sk#19)) -(38) Project [codegen id : 12] +(41) Project [codegen id : 12] Output [1]: [ca_address_sk#19] Input [2]: [ca_address_sk#19, ca_state#20] -(39) BroadcastExchange +(42) BroadcastExchange Input [1]: [ca_address_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(40) BroadcastHashJoin [codegen id : 13] +(43) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#19] Join condition: None -(41) Project [codegen id : 13] +(44) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] -(42) BroadcastExchange +(45) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(43) Scan parquet default.customer_demographics +(46) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(44) ColumnarToRow +(47) ColumnarToRow Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -(45) Filter +(48) Filter Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Condition : isnotnull(cd_demo_sk#23) -(46) BroadcastHashJoin [codegen id : 14] +(49) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#23] Join condition: None -(47) Project [codegen id : 14] +(50) Project [codegen id : 14] Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -(48) HashAggregate [codegen id : 14] +(51) HashAggregate [codegen id : 14] Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#29] Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] -(49) Exchange +(52) Exchange Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, 5), ENSURE_REQUIREMENTS, [id=#31] -(50) HashAggregate [codegen id : 15] +(53) HashAggregate [codegen id : 15] Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#32] Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#32 AS cnt1#33, cd_purchase_estimate#27, count(1)#32 AS cnt2#34, cd_credit_rating#28, count(1)#32 AS cnt3#35] -(51) TakeOrderedAndProject +(54) TakeOrderedAndProject Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -ReusedExchange (52) +ReusedExchange (55) -(52) ReusedExchange [Reuses operator id: 12] +(55) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt index dd7379392ec2f..9a333f419c399 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -33,11 +33,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #5 WholeStageCodegen (3) @@ -53,10 +54,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (7) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (11) @@ -66,10 +68,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (10) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [cs_ship_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index b5307abaf688c..5e074df039648 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -1,57 +1,60 @@ == Physical Plan == -TakeOrderedAndProject (46) -+- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (29) - : : +- * BroadcastHashJoin LeftAnti BuildRight (28) - : : :- * BroadcastHashJoin LeftAnti BuildRight (21) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (14) +TakeOrderedAndProject (49) ++- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (32) + : : +- * BroadcastHashJoin LeftAnti BuildRight (31) + : : :- * BroadcastHashJoin LeftAnti BuildRight (23) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (13) - : : : : +- * Project (12) - : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (10) - : : : : +- * Project (9) - : : : : +- * Filter (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.date_dim (6) - : : : +- BroadcastExchange (20) - : : : +- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * ColumnarToRow (16) - : : : : +- Scan parquet default.web_sales (15) - : : : +- ReusedExchange (17) - : : +- BroadcastExchange (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (24) - : +- BroadcastExchange (34) - : +- * Project (33) - : +- * Filter (32) - : +- * ColumnarToRow (31) - : +- Scan parquet default.customer_address (30) - +- BroadcastExchange (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet default.customer_demographics (37) + : : : : +- BroadcastExchange (14) + : : : : +- * Project (13) + : : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.store_sales (4) + : : : : +- BroadcastExchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.date_dim (7) + : : : +- BroadcastExchange (22) + : : : +- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Filter (18) + : : : : +- * ColumnarToRow (17) + : : : : +- Scan parquet default.web_sales (16) + : : : +- ReusedExchange (19) + : : +- BroadcastExchange (30) + : : +- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Filter (26) + : : : +- * ColumnarToRow (25) + : : : +- Scan parquet default.catalog_sales (24) + : : +- ReusedExchange (27) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- * Filter (35) + : +- * ColumnarToRow (34) + : +- Scan parquet default.customer_address (33) + +- BroadcastExchange (43) + +- * Filter (42) + +- * ColumnarToRow (41) + +- Scan parquet default.customer_demographics (40) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -59,215 +62,230 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) Scan parquet default.date_dim +(6) Filter [codegen id : 2] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Condition : isnotnull(ss_customer_sk#4) + +(7) Scan parquet default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 1] +(8) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -(8) Filter [codegen id : 1] +(9) Filter [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) -(9) Project [codegen id : 1] +(10) Project [codegen id : 1] Output [1]: [d_date_sk#7] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -(10) BroadcastExchange +(11) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] -(11) BroadcastHashJoin [codegen id : 2] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join condition: None -(12) Project [codegen id : 2] +(13) Project [codegen id : 2] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -(13) BroadcastExchange +(14) BroadcastExchange Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(14) BroadcastHashJoin [codegen id : 9] +(15) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join condition: None -(15) Scan parquet default.web_sales +(16) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 4] +(17) ColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] + +(18) Filter [codegen id : 4] Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +Condition : isnotnull(ws_bill_customer_sk#12) -(17) ReusedExchange [Reuses operator id: 10] +(19) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#7] -(18) BroadcastHashJoin [codegen id : 4] +(20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] Right keys [1]: [d_date_sk#7] Join condition: None -(19) Project [codegen id : 4] +(21) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12] Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] -(20) BroadcastExchange +(22) BroadcastExchange Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(21) BroadcastHashJoin [codegen id : 9] +(23) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#12] Join condition: None -(22) Scan parquet default.catalog_sales +(24) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 6] +(25) ColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] + +(26) Filter [codegen id : 6] Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_ship_customer_sk#15) -(24) ReusedExchange [Reuses operator id: 10] +(27) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#7] -(25) BroadcastHashJoin [codegen id : 6] +(28) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#7] Join condition: None -(26) Project [codegen id : 6] +(29) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#15] Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] -(27) BroadcastExchange +(30) BroadcastExchange Input [1]: [cs_ship_customer_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] -(28) BroadcastHashJoin [codegen id : 9] +(31) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#15] Join condition: None -(29) Project [codegen id : 9] +(32) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(30) Scan parquet default.customer_address +(33) Scan parquet default.customer_address Output [2]: [ca_address_sk#18, ca_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 7] +(34) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#18, ca_state#19] -(32) Filter [codegen id : 7] +(35) Filter [codegen id : 7] Input [2]: [ca_address_sk#18, ca_state#19] Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) -(33) Project [codegen id : 7] +(36) Project [codegen id : 7] Output [1]: [ca_address_sk#18] Input [2]: [ca_address_sk#18, ca_state#19] -(34) BroadcastExchange +(37) BroadcastExchange Input [1]: [ca_address_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] -(35) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#18] Join condition: None -(36) Project [codegen id : 9] +(39) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] -(37) Scan parquet default.customer_demographics +(40) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(38) ColumnarToRow [codegen id : 8] +(41) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -(39) Filter [codegen id : 8] +(42) Filter [codegen id : 8] Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Condition : isnotnull(cd_demo_sk#21) -(40) BroadcastExchange +(43) BroadcastExchange Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(41) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#21] Join condition: None -(42) Project [codegen id : 9] +(45) Project [codegen id : 9] Output [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -(43) HashAggregate [codegen id : 9] +(46) HashAggregate [codegen id : 9] Input [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#28] Results [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] -(44) Exchange +(47) Exchange Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] Arguments: hashpartitioning(cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, 5), ENSURE_REQUIREMENTS, [id=#30] -(45) HashAggregate [codegen id : 10] +(48) HashAggregate [codegen id : 10] Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#31] Results [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, count(1)#31 AS cnt1#32, cd_purchase_estimate#25, count(1)#31 AS cnt2#33, cd_credit_rating#26, count(1)#31 AS cnt3#34] -(46) TakeOrderedAndProject +(49) TakeOrderedAndProject Input [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#26 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (47) +ReusedExchange (50) -(47) ReusedExchange [Reuses operator id: 10] +(50) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#7] -Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index 957b9561b1752..7f39a7ff1796f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -22,11 +22,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -40,10 +41,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -51,10 +53,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [cs_ship_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index d31dbc3498ead..1e8bb4e77fcb4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -98,7 +98,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#8, s_county#9, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -106,7 +106,7 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Condition : isnotnull(s_store_sk#8) +Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_state#10)) (14) Scan parquet default.store_sales Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Condition : ((isnotnull(ranking#19) AND (ranking#19 <= 5)) AND isnotnull(s_state#16)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt index f541df9f07d00..8ee6469a58e56 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #4 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_state,s_store_sk] + Filter [s_store_sk,s_state] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_county,s_state] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 26fec145f4211..3f90f0bcea49f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -98,7 +98,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#8, s_county#9, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -106,7 +106,7 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Condition : isnotnull(s_store_sk#8) +Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_state#10)) (14) Scan parquet default.store_sales Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Condition : ((isnotnull(ranking#19) AND (ranking#19 <= 5)) AND isnotnull(s_state#16)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index 8a8866802dad6..d7b8e57cc8f4e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #4 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_state,s_store_sk] + Filter [s_store_sk,s_state] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_county,s_state] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index d006b61d20c33..7d458b55d672d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -1,72 +1,73 @@ == Physical Plan == -TakeOrderedAndProject (61) -+- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * HashAggregate (30) - : : +- Exchange (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) +TakeOrderedAndProject (62) ++- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_returns (1) - : : : +- BroadcastExchange (19) - : : : +- * Project (18) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (17) + : : : +- BroadcastExchange (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (18) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (14) - : : : :- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (13) - : : : +- * Project (12) - : : : +- * Filter (11) - : : : +- * ColumnarToRow (10) - : : : +- Scan parquet default.date_dim (9) - : : +- BroadcastExchange (25) - : : +- * Filter (24) - : : +- * ColumnarToRow (23) - : : +- Scan parquet default.item (22) - : +- BroadcastExchange (43) - : +- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Filter (33) - : : : +- * ColumnarToRow (32) - : : : +- Scan parquet default.catalog_returns (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- BroadcastExchange (58) - +- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet default.web_returns (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) + : : : +- BroadcastExchange (17) + : : : +- * Project (16) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (15) + : : : :- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.date_dim (7) + : : : +- BroadcastExchange (14) + : : : +- * Project (13) + : : : +- * Filter (12) + : : : +- * ColumnarToRow (11) + : : : +- Scan parquet default.date_dim (10) + : : +- BroadcastExchange (26) + : : +- * Filter (25) + : : +- * ColumnarToRow (24) + : : +- Scan parquet default.item (23) + : +- BroadcastExchange (44) + : +- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Filter (34) + : : : +- * ColumnarToRow (33) + : : : +- Scan parquet default.catalog_returns (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- BroadcastExchange (59) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Filter (49) + : : +- * ColumnarToRow (48) + : : +- Scan parquet default.web_returns (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) (1) Scan parquet default.store_returns Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(sr_item_sk)] ReadSchema: struct @@ -78,270 +79,295 @@ Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) (4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] +Output [2]: [d_date_sk#5, d_date#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (5) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#4, d_date#5] +Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 3] -Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Input [2]: [d_date_sk#5, d_date#6] +Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#6)) (7) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#6, d_week_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (8) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#6, d_week_seq#7] -(9) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +(9) Filter [codegen id : 2] +Input [2]: [d_date#6, d_week_seq#7] +Condition : (isnotnull(d_week_seq#7) AND isnotnull(d_date#6)) + +(10) Scan parquet default.date_dim +Output [2]: [d_date#6, d_week_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] +(11) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#6, d_week_seq#7] -(11) Filter [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] -Condition : cast(d_date#5 as string) IN (2000-06-30,2000-09-27,2000-11-17) +(12) Filter [codegen id : 1] +Input [2]: [d_date#6, d_week_seq#7] +Condition : (cast(d_date#6 as string) IN (2000-06-30,2000-09-27,2000-11-17) AND isnotnull(d_week_seq#7)) -(12) Project [codegen id : 1] -Output [1]: [d_week_seq#6 AS d_week_seq#6#7] -Input [2]: [d_date#5, d_week_seq#6] +(13) Project [codegen id : 1] +Output [1]: [d_week_seq#7 AS d_week_seq#7#8] +Input [2]: [d_date#6, d_week_seq#7] -(13) BroadcastExchange -Input [1]: [d_week_seq#6#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +(14) BroadcastExchange +Input [1]: [d_week_seq#7#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] -(14) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#6] -Right keys [1]: [d_week_seq#6#7] +(15) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [d_week_seq#7] +Right keys [1]: [d_week_seq#7#8] Join condition: None -(15) Project [codegen id : 2] -Output [1]: [d_date#5 AS d_date#5#9] -Input [2]: [d_date#5, d_week_seq#6] +(16) Project [codegen id : 2] +Output [1]: [d_date#6 AS d_date#6#10] +Input [2]: [d_date#6, d_week_seq#7] -(16) BroadcastExchange -Input [1]: [d_date#5#9] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] +(17) BroadcastExchange +Input [1]: [d_date#6#10] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#11] -(17) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#9] +(18) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date#6] +Right keys [1]: [d_date#6#10] Join condition: None -(18) Project [codegen id : 3] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +(19) Project [codegen id : 3] +Output [1]: [d_date_sk#5] +Input [2]: [d_date_sk#5, d_date#6] -(19) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +(20) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] -(20) BroadcastHashJoin [codegen id : 5] +(21) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] -Right keys [1]: [cast(d_date_sk#4 as bigint)] +Right keys [1]: [cast(d_date_sk#5 as bigint)] Join condition: None -(21) Project [codegen id : 5] +(22) Project [codegen id : 5] Output [2]: [sr_item_sk#1, sr_return_quantity#2] -Input [4]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, d_date_sk#4] +Input [4]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, d_date_sk#5] -(22) Scan parquet default.item -Output [2]: [i_item_sk#12, i_item_id#13] +(23) Scan parquet default.item +Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#12, i_item_id#13] +(24) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#13, i_item_id#14] -(24) Filter [codegen id : 4] -Input [2]: [i_item_sk#12, i_item_id#13] -Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) +(25) Filter [codegen id : 4] +Input [2]: [i_item_sk#13, i_item_id#14] +Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) -(25) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] +(26) BroadcastExchange +Input [2]: [i_item_sk#13, i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] -(26) BroadcastHashJoin [codegen id : 5] +(27) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#1] -Right keys [1]: [cast(i_item_sk#12 as bigint)] +Right keys [1]: [cast(i_item_sk#13 as bigint)] Join condition: None -(27) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#13] -Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#12, i_item_id#13] +(28) Project [codegen id : 5] +Output [2]: [sr_return_quantity#2, i_item_id#14] +Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#13, i_item_id#14] -(28) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#2, i_item_id#13] -Keys [1]: [i_item_id#13] +(29) HashAggregate [codegen id : 5] +Input [2]: [sr_return_quantity#2, i_item_id#14] +Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#13, sum#16] +Aggregate Attributes [1]: [sum#16] +Results [2]: [i_item_id#14, sum#17] -(29) Exchange -Input [2]: [i_item_id#13, sum#16] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] +(30) Exchange +Input [2]: [i_item_id#14, sum#17] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#18] -(30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#13, sum#16] -Keys [1]: [i_item_id#13] +(31) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#14, sum#17] +Keys [1]: [i_item_id#14] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] -Results [2]: [i_item_id#13 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] +Results [2]: [i_item_id#14 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] -(31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +(32) Scan parquet default.catalog_returns +Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(cr_returned_date_sk#24 IN dynamicpruning#25)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +(33) ColumnarToRow [codegen id : 10] +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] -(33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] -Condition : isnotnull(cr_item_sk#21) +(34) Filter [codegen id : 10] +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Condition : isnotnull(cr_item_sk#22) -(34) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#4] +(35) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#5] -(35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#4] +(36) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#24] +Right keys [1]: [d_date_sk#5] Join condition: None -(36) Project [codegen id : 10] -Output [2]: [cr_item_sk#21, cr_return_quantity#22] -Input [4]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, d_date_sk#4] +(37) Project [codegen id : 10] +Output [2]: [cr_item_sk#22, cr_return_quantity#23] +Input [4]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, d_date_sk#5] -(37) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#12, i_item_id#13] +(38) ReusedExchange [Reuses operator id: 26] +Output [2]: [i_item_sk#13, i_item_id#14] -(38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#21] -Right keys [1]: [i_item_sk#12] +(39) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_item_sk#22] +Right keys [1]: [i_item_sk#13] Join condition: None -(39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#22, i_item_id#13] -Input [4]: [cr_item_sk#21, cr_return_quantity#22, i_item_sk#12, i_item_id#13] - -(40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#22, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum#24] -Results [2]: [i_item_id#13, sum#25] - -(41) Exchange -Input [2]: [i_item_id#13, sum#25] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#26] - -(42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#13, sum#25] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] -Results [2]: [i_item_id#13 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] - -(43) BroadcastExchange -Input [2]: [item_id#28, cr_item_qty#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] - -(44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#28] +(40) Project [codegen id : 10] +Output [2]: [cr_return_quantity#23, i_item_id#14] +Input [4]: [cr_item_sk#22, cr_return_quantity#23, i_item_sk#13, i_item_id#14] + +(41) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#23, i_item_id#14] +Keys [1]: [i_item_id#14] +Functions [1]: [partial_sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum#26] +Results [2]: [i_item_id#14, sum#27] + +(42) Exchange +Input [2]: [i_item_id#14, sum#27] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#28] + +(43) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#14, sum#27] +Keys [1]: [i_item_id#14] +Functions [1]: [sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum(cr_return_quantity#23)#29] +Results [2]: [i_item_id#14 AS item_id#30, sum(cr_return_quantity#23)#29 AS cr_item_qty#31] + +(44) BroadcastExchange +Input [2]: [item_id#30, cr_item_qty#31] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] + +(45) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#30] Join condition: None -(45) Project [codegen id : 18] -Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] -Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] +(46) Project [codegen id : 18] +Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#31] +Input [4]: [item_id#20, sr_item_qty#21, item_id#30, cr_item_qty#31] -(46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +(47) Scan parquet default.web_returns +Output [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#35), dynamicpruningexpression(wr_returned_date_sk#35 IN dynamicpruning#4)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +(48) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] -(48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -Condition : isnotnull(wr_item_sk#31) +(49) Filter [codegen id : 16] +Input [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] +Condition : isnotnull(wr_item_sk#33) -(49) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#4] +(50) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#5] -(50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#33] -Right keys [1]: [cast(d_date_sk#4 as bigint)] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#35] +Right keys [1]: [cast(d_date_sk#5 as bigint)] Join condition: None -(51) Project [codegen id : 16] -Output [2]: [wr_item_sk#31, wr_return_quantity#32] -Input [4]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, d_date_sk#4] +(52) Project [codegen id : 16] +Output [2]: [wr_item_sk#33, wr_return_quantity#34] +Input [4]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35, d_date_sk#5] -(52) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#12, i_item_id#13] +(53) ReusedExchange [Reuses operator id: 26] +Output [2]: [i_item_sk#13, i_item_id#14] -(53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#31] -Right keys [1]: [cast(i_item_sk#12 as bigint)] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#33] +Right keys [1]: [cast(i_item_sk#13 as bigint)] Join condition: None -(54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#32, i_item_id#13] -Input [4]: [wr_item_sk#31, wr_return_quantity#32, i_item_sk#12, i_item_id#13] - -(55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#32, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#13, sum#35] - -(56) Exchange -Input [2]: [i_item_id#13, sum#35] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#36] - -(57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#13, sum#35] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] -Results [2]: [i_item_id#13 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] - -(58) BroadcastExchange -Input [2]: [item_id#38, wr_item_qty#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] - -(59) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#38] +(55) Project [codegen id : 16] +Output [2]: [wr_return_quantity#34, i_item_id#14] +Input [4]: [wr_item_sk#33, wr_return_quantity#34, i_item_sk#13, i_item_id#14] + +(56) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#34, i_item_id#14] +Keys [1]: [i_item_id#14] +Functions [1]: [partial_sum(wr_return_quantity#34)] +Aggregate Attributes [1]: [sum#36] +Results [2]: [i_item_id#14, sum#37] + +(57) Exchange +Input [2]: [i_item_id#14, sum#37] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#38] + +(58) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#14, sum#37] +Keys [1]: [i_item_id#14] +Functions [1]: [sum(wr_return_quantity#34)] +Aggregate Attributes [1]: [sum(wr_return_quantity#34)#39] +Results [2]: [i_item_id#14 AS item_id#40, sum(wr_return_quantity#34)#39 AS wr_item_qty#41] + +(59) BroadcastExchange +Input [2]: [item_id#40, wr_item_qty#41] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] + +(60) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#40] Join condition: None -(60) Project [codegen id : 18] -Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] -Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] +(61) Project [codegen id : 18] +Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS sr_dev#43, cr_item_qty#31, (((cast(cr_item_qty#31 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS cr_dev#44, wr_item_qty#41, (((cast(wr_item_qty#41 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS wr_dev#45, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#46] +Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#31, item_id#40, wr_item_qty#41] + +(62) TakeOrderedAndProject +Input [8]: [item_id#20, sr_item_qty#21, sr_dev#43, cr_item_qty#31, cr_dev#44, wr_item_qty#41, wr_dev#45, average#46] +Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#43, cr_item_qty#31, cr_dev#44, wr_item_qty#41, wr_dev#45, average#46] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 +ReusedExchange (63) + + +(63) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#5] + +Subquery:2 Hosting operator id = 32 Hosting Expression = cr_returned_date_sk#24 IN dynamicpruning#25 +ReusedExchange (64) + + +(64) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#5] + +Subquery:3 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#35 IN dynamicpruning#4 -(61) TakeOrderedAndProject -Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] -Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt index 135e7a4760013..237c969825d92 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt @@ -17,12 +17,14 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #2 WholeStageCodegen (3) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date,d_date_sk] + Filter [d_date_sk,d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] @@ -31,7 +33,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (2) Project [d_date] BroadcastHashJoin [d_week_seq,d_week_seq] - Filter [d_date,d_week_seq] + Filter [d_week_seq,d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] @@ -66,6 +68,8 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter @@ -86,6 +90,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index c926ab7e3ccb6..f33bf1d745219 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,14 +1,14 @@ == Physical Plan == -TakeOrderedAndProject (61) -+- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * HashAggregate (30) - : : +- Exchange (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) +TakeOrderedAndProject (62) ++- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) : : :- * Project (9) : : : +- * BroadcastHashJoin Inner BuildRight (8) : : : :- * Filter (3) @@ -18,55 +18,56 @@ TakeOrderedAndProject (61) : : : +- * Filter (6) : : : +- * ColumnarToRow (5) : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin LeftSemi BuildRight (24) : : :- * Filter (12) : : : +- * ColumnarToRow (11) : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- * BroadcastHashJoin LeftSemi BuildRight (20) - : : :- * ColumnarToRow (14) - : : : +- Scan parquet default.date_dim (13) - : : +- BroadcastExchange (19) - : : +- * Project (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.date_dim (15) - : +- BroadcastExchange (43) - : +- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Filter (33) - : : : +- * ColumnarToRow (32) - : : : +- Scan parquet default.catalog_returns (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- BroadcastExchange (58) - +- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet default.web_returns (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin LeftSemi BuildRight (21) + : : :- * Filter (15) + : : : +- * ColumnarToRow (14) + : : : +- Scan parquet default.date_dim (13) + : : +- BroadcastExchange (20) + : : +- * Project (19) + : : +- * Filter (18) + : : +- * ColumnarToRow (17) + : : +- Scan parquet default.date_dim (16) + : +- BroadcastExchange (44) + : +- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Filter (34) + : : : +- * ColumnarToRow (33) + : : : +- Scan parquet default.catalog_returns (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- BroadcastExchange (59) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Filter (49) + : : +- * ColumnarToRow (48) + : : +- Scan parquet default.web_returns (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) (1) Scan parquet default.store_returns Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(sr_item_sk)] ReadSchema: struct @@ -78,270 +79,295 @@ Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) (4) Scan parquet default.item -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#5, i_item_id#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#4, i_item_id#5] +Input [2]: [i_item_sk#5, i_item_id#6] (6) Filter [codegen id : 1] -Input [2]: [i_item_sk#4, i_item_id#5] -Condition : (isnotnull(i_item_sk#4) AND isnotnull(i_item_id#5)) +Input [2]: [i_item_sk#5, i_item_id#6] +Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) (7) BroadcastExchange -Input [2]: [i_item_sk#4, i_item_id#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#6] +Input [2]: [i_item_sk#5, i_item_id#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] (8) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#1] -Right keys [1]: [cast(i_item_sk#4 as bigint)] +Right keys [1]: [cast(i_item_sk#5 as bigint)] Join condition: None (9) Project [codegen id : 5] -Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#5] -Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#4, i_item_id#5] +Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] +Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#6] (10) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] +Output [2]: [d_date_sk#8, d_date#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (11) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#7, d_date#8] +Input [2]: [d_date_sk#8, d_date#9] (12) Filter [codegen id : 4] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Input [2]: [d_date_sk#8, d_date#9] +Condition : (isnotnull(d_date_sk#8) AND isnotnull(d_date#9)) (13) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#9, d_week_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#9, d_week_seq#10] -(15) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +(15) Filter [codegen id : 3] +Input [2]: [d_date#9, d_week_seq#10] +Condition : (isnotnull(d_week_seq#10) AND isnotnull(d_date#9)) + +(16) Scan parquet default.date_dim +Output [2]: [d_date#9, d_week_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] +(17) ColumnarToRow [codegen id : 2] +Input [2]: [d_date#9, d_week_seq#10] -(17) Filter [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] -Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) +(18) Filter [codegen id : 2] +Input [2]: [d_date#9, d_week_seq#10] +Condition : (cast(d_date#9 as string) IN (2000-06-30,2000-09-27,2000-11-17) AND isnotnull(d_week_seq#10)) -(18) Project [codegen id : 2] -Output [1]: [d_week_seq#9 AS d_week_seq#9#10] -Input [2]: [d_date#8, d_week_seq#9] +(19) Project [codegen id : 2] +Output [1]: [d_week_seq#10 AS d_week_seq#10#11] +Input [2]: [d_date#9, d_week_seq#10] -(19) BroadcastExchange -Input [1]: [d_week_seq#9#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +(20) BroadcastExchange +Input [1]: [d_week_seq#10#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_week_seq#9] -Right keys [1]: [d_week_seq#9#10] +(21) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_week_seq#10] +Right keys [1]: [d_week_seq#10#11] Join condition: None -(21) Project [codegen id : 3] -Output [1]: [d_date#8 AS d_date#8#12] -Input [2]: [d_date#8, d_week_seq#9] +(22) Project [codegen id : 3] +Output [1]: [d_date#9 AS d_date#9#13] +Input [2]: [d_date#9, d_week_seq#10] -(22) BroadcastExchange -Input [1]: [d_date#8#12] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] +(23) BroadcastExchange +Input [1]: [d_date#9#13] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#14] -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#12] +(24) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [d_date#9] +Right keys [1]: [d_date#9#13] Join condition: None -(24) Project [codegen id : 4] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] +(25) Project [codegen id : 4] +Output [1]: [d_date_sk#8] +Input [2]: [d_date_sk#8, d_date#9] -(25) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +(26) BroadcastExchange +Input [1]: [d_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] -(26) BroadcastHashJoin [codegen id : 5] +(27) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] -Right keys [1]: [cast(d_date_sk#7 as bigint)] +Right keys [1]: [cast(d_date_sk#8 as bigint)] Join condition: None -(27) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#5] -Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#5, d_date_sk#7] +(28) Project [codegen id : 5] +Output [2]: [sr_return_quantity#2, i_item_id#6] +Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6, d_date_sk#8] -(28) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#2, i_item_id#5] -Keys [1]: [i_item_id#5] +(29) HashAggregate [codegen id : 5] +Input [2]: [sr_return_quantity#2, i_item_id#6] +Keys [1]: [i_item_id#6] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#5, sum#16] +Aggregate Attributes [1]: [sum#16] +Results [2]: [i_item_id#6, sum#17] -(29) Exchange -Input [2]: [i_item_id#5, sum#16] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#17] +(30) Exchange +Input [2]: [i_item_id#6, sum#17] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#18] -(30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#5, sum#16] -Keys [1]: [i_item_id#5] +(31) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#6, sum#17] +Keys [1]: [i_item_id#6] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] -Results [2]: [i_item_id#5 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] +Results [2]: [i_item_id#6 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] -(31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +(32) Scan parquet default.catalog_returns +Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(cr_returned_date_sk#24 IN dynamicpruning#25)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +(33) ColumnarToRow [codegen id : 10] +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] -(33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] -Condition : isnotnull(cr_item_sk#21) +(34) Filter [codegen id : 10] +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Condition : isnotnull(cr_item_sk#22) -(34) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +(35) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#5, i_item_id#6] -(35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#21] -Right keys [1]: [i_item_sk#4] +(36) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_item_sk#22] +Right keys [1]: [i_item_sk#5] Join condition: None -(36) Project [codegen id : 10] -Output [3]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5] -Input [5]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, i_item_sk#4, i_item_id#5] +(37) Project [codegen id : 10] +Output [3]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#6] +Input [5]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, i_item_sk#5, i_item_id#6] -(37) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#7] +(38) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#8] -(38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#7] +(39) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#24] +Right keys [1]: [d_date_sk#8] Join condition: None -(39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#22, i_item_id#5] -Input [4]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5, d_date_sk#7] - -(40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#22, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum#24] -Results [2]: [i_item_id#5, sum#25] - -(41) Exchange -Input [2]: [i_item_id#5, sum#25] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#26] - -(42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#5, sum#25] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] -Results [2]: [i_item_id#5 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] - -(43) BroadcastExchange -Input [2]: [item_id#28, cr_item_qty#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] - -(44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#28] +(40) Project [codegen id : 10] +Output [2]: [cr_return_quantity#23, i_item_id#6] +Input [4]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#6, d_date_sk#8] + +(41) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#23, i_item_id#6] +Keys [1]: [i_item_id#6] +Functions [1]: [partial_sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum#26] +Results [2]: [i_item_id#6, sum#27] + +(42) Exchange +Input [2]: [i_item_id#6, sum#27] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#28] + +(43) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#6, sum#27] +Keys [1]: [i_item_id#6] +Functions [1]: [sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum(cr_return_quantity#23)#29] +Results [2]: [i_item_id#6 AS item_id#30, sum(cr_return_quantity#23)#29 AS cr_item_qty#31] + +(44) BroadcastExchange +Input [2]: [item_id#30, cr_item_qty#31] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] + +(45) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#30] Join condition: None -(45) Project [codegen id : 18] -Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] -Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] +(46) Project [codegen id : 18] +Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#31] +Input [4]: [item_id#20, sr_item_qty#21, item_id#30, cr_item_qty#31] -(46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +(47) Scan parquet default.web_returns +Output [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#35), dynamicpruningexpression(wr_returned_date_sk#35 IN dynamicpruning#4)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +(48) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] -(48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -Condition : isnotnull(wr_item_sk#31) +(49) Filter [codegen id : 16] +Input [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] +Condition : isnotnull(wr_item_sk#33) -(49) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +(50) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#5, i_item_id#6] -(50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#31] -Right keys [1]: [cast(i_item_sk#4 as bigint)] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#33] +Right keys [1]: [cast(i_item_sk#5 as bigint)] Join condition: None -(51) Project [codegen id : 16] -Output [3]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5] -Input [5]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, i_item_sk#4, i_item_id#5] +(52) Project [codegen id : 16] +Output [3]: [wr_return_quantity#34, wr_returned_date_sk#35, i_item_id#6] +Input [5]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35, i_item_sk#5, i_item_id#6] -(52) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#7] +(53) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#8] -(53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#33] -Right keys [1]: [cast(d_date_sk#7 as bigint)] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#35] +Right keys [1]: [cast(d_date_sk#8 as bigint)] Join condition: None -(54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#32, i_item_id#5] -Input [4]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5, d_date_sk#7] - -(55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#32, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#5, sum#35] - -(56) Exchange -Input [2]: [i_item_id#5, sum#35] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#36] - -(57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#5, sum#35] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] -Results [2]: [i_item_id#5 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] - -(58) BroadcastExchange -Input [2]: [item_id#38, wr_item_qty#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] - -(59) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#38] +(55) Project [codegen id : 16] +Output [2]: [wr_return_quantity#34, i_item_id#6] +Input [4]: [wr_return_quantity#34, wr_returned_date_sk#35, i_item_id#6, d_date_sk#8] + +(56) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#34, i_item_id#6] +Keys [1]: [i_item_id#6] +Functions [1]: [partial_sum(wr_return_quantity#34)] +Aggregate Attributes [1]: [sum#36] +Results [2]: [i_item_id#6, sum#37] + +(57) Exchange +Input [2]: [i_item_id#6, sum#37] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#38] + +(58) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#6, sum#37] +Keys [1]: [i_item_id#6] +Functions [1]: [sum(wr_return_quantity#34)] +Aggregate Attributes [1]: [sum(wr_return_quantity#34)#39] +Results [2]: [i_item_id#6 AS item_id#40, sum(wr_return_quantity#34)#39 AS wr_item_qty#41] + +(59) BroadcastExchange +Input [2]: [item_id#40, wr_item_qty#41] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] + +(60) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#40] Join condition: None -(60) Project [codegen id : 18] -Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] -Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] +(61) Project [codegen id : 18] +Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS sr_dev#43, cr_item_qty#31, (((cast(cr_item_qty#31 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS cr_dev#44, wr_item_qty#41, (((cast(wr_item_qty#41 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS wr_dev#45, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#46] +Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#31, item_id#40, wr_item_qty#41] + +(62) TakeOrderedAndProject +Input [8]: [item_id#20, sr_item_qty#21, sr_dev#43, cr_item_qty#31, cr_dev#44, wr_item_qty#41, wr_dev#45, average#46] +Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#43, cr_item_qty#31, cr_dev#44, wr_item_qty#41, wr_dev#45, average#46] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 +ReusedExchange (63) + + +(63) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#8] + +Subquery:2 Hosting operator id = 32 Hosting Expression = cr_returned_date_sk#24 IN dynamicpruning#25 +ReusedExchange (64) + + +(64) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#8] + +Subquery:3 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#35 IN dynamicpruning#4 -(61) TakeOrderedAndProject -Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] -Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index a47eb7cb8cfb8..fc2b465906e4e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -17,19 +17,21 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #2 + BroadcastExchange #3 WholeStageCodegen (1) Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #3 + BroadcastExchange #2 WholeStageCodegen (4) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date,d_date_sk] + Filter [d_date_sk,d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] @@ -38,7 +40,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (3) Project [d_date] BroadcastHashJoin [d_week_seq,d_week_seq] - Filter [d_date,d_week_seq] + Filter [d_week_seq,d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] @@ -66,10 +68,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 + ReusedExchange [i_item_sk,i_item_id] #3 InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #8 WholeStageCodegen (17) @@ -86,7 +90,8 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #2 + ReusedExchange [i_item_sk,i_item_id] #3 InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #2 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt index aec81c0d5c7aa..19cbce69c0768 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- Exchange (22) - +- * HashAggregate (21) - +- * Project (20) - +- * SortMergeJoin Inner (19) +TakeOrderedAndProject (25) ++- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * SortMergeJoin Inner (20) :- * Sort (13) : +- Exchange (12) : +- * Project (11) @@ -18,11 +18,12 @@ TakeOrderedAndProject (24) : +- * Filter (7) : +- * ColumnarToRow (6) : +- Scan parquet default.reason (5) - +- * Sort (18) - +- Exchange (17) - +- * Project (16) - +- * ColumnarToRow (15) - +- Scan parquet default.store_sales (14) + +- * Sort (19) + +- Exchange (18) + +- * Project (17) + +- * Filter (16) + +- * ColumnarToRow (15) + +- Scan parquet default.store_sales (14) (1) Scan parquet default.store_returns @@ -47,7 +48,7 @@ Input [5]: [sr_item_sk#1, sr_reason_sk#2, sr_ticket_number#3, sr_return_quantity Output [2]: [r_reason_sk#6, r_reason_desc#7] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28), IsNotNull(r_reason_sk)] +PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -55,7 +56,7 @@ Input [2]: [r_reason_sk#6, r_reason_desc#7] (7) Filter [codegen id : 1] Input [2]: [r_reason_sk#6, r_reason_desc#7] -Condition : ((isnotnull(r_reason_desc#7) AND (r_reason_desc#7 = reason 28)) AND isnotnull(r_reason_sk#6)) +Condition : ((isnotnull(r_reason_desc#7) AND (r_reason_desc#7 = reason 28 )) AND isnotnull(r_reason_sk#6)) (8) Project [codegen id : 1] Output [1]: [r_reason_sk#6] @@ -86,51 +87,56 @@ Arguments: [sr_item_sk#1 ASC NULLS FIRST, sr_ticket_number#3 ASC NULLS FIRST], f Output [6]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14, ss_sold_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] Input [6]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14, ss_sold_date_sk#15] -(16) Project [codegen id : 4] +(16) Filter [codegen id : 4] +Input [6]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14, ss_sold_date_sk#15] +Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_ticket_number#12)) + +(17) Project [codegen id : 4] Output [5]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14] Input [6]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14, ss_sold_date_sk#15] -(17) Exchange +(18) Exchange Input [5]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14] Arguments: hashpartitioning(cast(ss_item_sk#10 as bigint), cast(ss_ticket_number#12 as bigint), 5), ENSURE_REQUIREMENTS, [id=#16] -(18) Sort [codegen id : 5] +(19) Sort [codegen id : 5] Input [5]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14] Arguments: [cast(ss_item_sk#10 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#12 as bigint) ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 6] +(20) SortMergeJoin [codegen id : 6] Left keys [2]: [sr_item_sk#1, sr_ticket_number#3] Right keys [2]: [cast(ss_item_sk#10 as bigint), cast(ss_ticket_number#12 as bigint)] Join condition: None -(20) Project [codegen id : 6] +(21) Project [codegen id : 6] Output [2]: [ss_customer_sk#11, CASE WHEN isnotnull(sr_return_quantity#4) THEN CheckOverflow((promote_precision(cast(cast((ss_quantity#13 - sr_return_quantity#4) as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#14 as decimal(12,2)))), DecimalType(18,2), true) ELSE CheckOverflow((promote_precision(cast(cast(ss_quantity#13 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#14 as decimal(12,2)))), DecimalType(18,2), true) END AS act_sales#17] Input [8]: [sr_item_sk#1, sr_ticket_number#3, sr_return_quantity#4, ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14] -(21) HashAggregate [codegen id : 6] +(22) HashAggregate [codegen id : 6] Input [2]: [ss_customer_sk#11, act_sales#17] Keys [1]: [ss_customer_sk#11] Functions [1]: [partial_sum(act_sales#17)] Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [3]: [ss_customer_sk#11, sum#20, isEmpty#21] -(22) Exchange +(23) Exchange Input [3]: [ss_customer_sk#11, sum#20, isEmpty#21] Arguments: hashpartitioning(ss_customer_sk#11, 5), ENSURE_REQUIREMENTS, [id=#22] -(23) HashAggregate [codegen id : 7] +(24) HashAggregate [codegen id : 7] Input [3]: [ss_customer_sk#11, sum#20, isEmpty#21] Keys [1]: [ss_customer_sk#11] Functions [1]: [sum(act_sales#17)] Aggregate Attributes [1]: [sum(act_sales#17)#23] Results [2]: [ss_customer_sk#11, sum(act_sales#17)#23 AS sumsales#24] -(24) TakeOrderedAndProject +(25) TakeOrderedAndProject Input [2]: [ss_customer_sk#11, sumsales#24] Arguments: 100, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#11 ASC NULLS FIRST], [ss_customer_sk#11, sumsales#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt index 17464e295b1dd..fcebeb98855be 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt @@ -35,6 +35,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] Exchange [ss_item_sk,ss_ticket_number] #4 WholeStageCodegen (4) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + Filter [ss_item_sk,ss_ticket_number] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index 1875c5418a890..cd69276ebcb4a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -1,136 +1,142 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- Exchange (22) - +- * HashAggregate (21) - +- * Project (20) - +- * BroadcastHashJoin Inner BuildRight (19) - :- * Project (13) - : +- * SortMergeJoin Inner (12) - : :- * Sort (5) - : : +- Exchange (4) - : : +- * Project (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- * Sort (11) - : +- Exchange (10) - : +- * Project (9) - : +- * Filter (8) - : +- * ColumnarToRow (7) - : +- Scan parquet default.store_returns (6) - +- BroadcastExchange (18) - +- * Project (17) - +- * Filter (16) - +- * ColumnarToRow (15) - +- Scan parquet default.reason (14) +TakeOrderedAndProject (25) ++- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildRight (20) + :- * Project (14) + : +- * SortMergeJoin Inner (13) + : :- * Sort (6) + : : +- Exchange (5) + : : +- * Project (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.store_sales (1) + : +- * Sort (12) + : +- Exchange (11) + : +- * Project (10) + : +- * Filter (9) + : +- * ColumnarToRow (8) + : +- Scan parquet default.store_returns (7) + +- BroadcastExchange (19) + +- * Project (18) + +- * Filter (17) + +- * ColumnarToRow (16) + +- Scan parquet default.reason (15) (1) Scan parquet default.store_sales Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -(3) Project [codegen id : 1] +(3) Filter [codegen id : 1] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] +Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#3)) + +(4) Project [codegen id : 1] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -(4) Exchange +(5) Exchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] Arguments: hashpartitioning(cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#3 as bigint), 5), ENSURE_REQUIREMENTS, [id=#7] -(5) Sort [codegen id : 2] +(6) Sort [codegen id : 2] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] Arguments: [cast(ss_item_sk#1 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#3 as bigint) ASC NULLS FIRST], false, 0 -(6) Scan parquet default.store_returns +(7) Scan parquet default.store_returns Output [5]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 3] +(8) ColumnarToRow [codegen id : 3] Input [5]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -(8) Filter [codegen id : 3] +(9) Filter [codegen id : 3] Input [5]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Condition : ((isnotnull(sr_item_sk#8) AND isnotnull(sr_ticket_number#10)) AND isnotnull(sr_reason_sk#9)) -(9) Project [codegen id : 3] +(10) Project [codegen id : 3] Output [4]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11] Input [5]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -(10) Exchange +(11) Exchange Input [4]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11] Arguments: hashpartitioning(sr_item_sk#8, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [id=#13] -(11) Sort [codegen id : 4] +(12) Sort [codegen id : 4] Input [4]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11] Arguments: [sr_item_sk#8 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 6] +(13) SortMergeJoin [codegen id : 6] Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#3 as bigint)] Right keys [2]: [sr_item_sk#8, sr_ticket_number#10] Join condition: None -(13) Project [codegen id : 6] +(14) Project [codegen id : 6] Output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#9, sr_return_quantity#11] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11] -(14) Scan parquet default.reason +(15) Scan parquet default.reason Output [2]: [r_reason_sk#14, r_reason_desc#15] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28), IsNotNull(r_reason_sk)] +PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 5] +(16) ColumnarToRow [codegen id : 5] Input [2]: [r_reason_sk#14, r_reason_desc#15] -(16) Filter [codegen id : 5] +(17) Filter [codegen id : 5] Input [2]: [r_reason_sk#14, r_reason_desc#15] -Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28)) AND isnotnull(r_reason_sk#14)) +Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28 )) AND isnotnull(r_reason_sk#14)) -(17) Project [codegen id : 5] +(18) Project [codegen id : 5] Output [1]: [r_reason_sk#14] Input [2]: [r_reason_sk#14, r_reason_desc#15] -(18) BroadcastExchange +(19) BroadcastExchange Input [1]: [r_reason_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(19) BroadcastHashJoin [codegen id : 6] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_reason_sk#9] Right keys [1]: [cast(r_reason_sk#14 as bigint)] Join condition: None -(20) Project [codegen id : 6] +(21) Project [codegen id : 6] Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#11) THEN CheckOverflow((promote_precision(cast(cast((ss_quantity#4 - sr_return_quantity#11) as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#5 as decimal(12,2)))), DecimalType(18,2), true) ELSE CheckOverflow((promote_precision(cast(cast(ss_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#5 as decimal(12,2)))), DecimalType(18,2), true) END AS act_sales#17] Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#9, sr_return_quantity#11, r_reason_sk#14] -(21) HashAggregate [codegen id : 6] +(22) HashAggregate [codegen id : 6] Input [2]: [ss_customer_sk#2, act_sales#17] Keys [1]: [ss_customer_sk#2] Functions [1]: [partial_sum(act_sales#17)] Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [3]: [ss_customer_sk#2, sum#20, isEmpty#21] -(22) Exchange +(23) Exchange Input [3]: [ss_customer_sk#2, sum#20, isEmpty#21] Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#22] -(23) HashAggregate [codegen id : 7] +(24) HashAggregate [codegen id : 7] Input [3]: [ss_customer_sk#2, sum#20, isEmpty#21] Keys [1]: [ss_customer_sk#2] Functions [1]: [sum(act_sales#17)] Aggregate Attributes [1]: [sum(act_sales#17)#23] Results [2]: [ss_customer_sk#2, sum(act_sales#17)#23 AS sumsales#24] -(24) TakeOrderedAndProject +(25) TakeOrderedAndProject Input [2]: [ss_customer_sk#2, sumsales#24] Arguments: 100, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index 0d9aec90a2da4..c6441ac0fc899 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -16,9 +16,10 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] Exchange [ss_item_sk,ss_ticket_number] #2 WholeStageCodegen (1) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + Filter [ss_item_sk,ss_ticket_number] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) Sort [sr_item_sk,sr_ticket_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index bee7110ecd6dd..eb516affccfed 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -1,60 +1,62 @@ == Physical Plan == -* Sort (49) -+- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- SortMergeJoin LeftAnti (21) - : : : :- * Sort (15) - : : : : +- Exchange (14) - : : : : +- * Project (13) - : : : : +- SortMergeJoin LeftSemi (12) +* Sort (51) ++- * HashAggregate (50) + +- Exchange (49) + +- * HashAggregate (48) + +- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- SortMergeJoin LeftAnti (23) + : : : :- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Project (14) + : : : : +- SortMergeJoin LeftSemi (13) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.web_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * Project (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.web_sales (7) - : : : +- * Sort (20) - : : : +- Exchange (19) - : : : +- * Project (18) - : : : +- * ColumnarToRow (17) - : : : +- Scan parquet default.web_returns (16) - : : +- BroadcastExchange (26) - : : +- * Project (25) - : : +- * Filter (24) - : : +- * ColumnarToRow (23) - : : +- Scan parquet default.customer_address (22) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- * ColumnarToRow (30) - : +- Scan parquet default.web_site (29) - +- BroadcastExchange (40) - +- * Project (39) - +- * Filter (38) - +- * ColumnarToRow (37) - +- Scan parquet default.date_dim (36) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.web_sales (7) + : : : +- * Sort (22) + : : : +- Exchange (21) + : : : +- * Project (20) + : : : +- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.web_returns (17) + : : +- BroadcastExchange (28) + : : +- * Project (27) + : : +- * Filter (26) + : : +- * ColumnarToRow (25) + : : +- Scan parquet default.customer_address (24) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.web_site (31) + +- BroadcastExchange (42) + +- * Project (41) + +- * Filter (40) + +- * ColumnarToRow (39) + +- Scan parquet default.date_dim (38) (1) Scan parquet default.web_sales Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -62,7 +64,7 @@ Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (3) Filter [codegen id : 1] Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) +Condition : ((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#5)) AND isnotnull(ws_warehouse_sk#4)) (4) Project [codegen id : 1] Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -80,196 +82,206 @@ Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -(9) Project [codegen id : 3] +(9) Filter [codegen id : 3] +Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Condition : (isnotnull(ws_order_number#5) AND isnotnull(ws_warehouse_sk#4)) + +(10) Project [codegen id : 3] Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -(10) Exchange +(11) Exchange Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] -(11) Sort [codegen id : 4] +(12) Sort [codegen id : 4] Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin +(13) SortMergeJoin Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#5#11] Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) -(13) Project [codegen id : 5] +(14) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(14) Exchange +(15) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] -(15) Sort [codegen id : 6] +(16) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 -(16) Scan parquet default.web_returns +(17) Scan parquet default.web_returns Output [2]: [wr_order_number#14, wr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 7] +(18) ColumnarToRow [codegen id : 7] +Input [2]: [wr_order_number#14, wr_returned_date_sk#15] + +(19) Filter [codegen id : 7] Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Condition : isnotnull(wr_order_number#14) -(18) Project [codegen id : 7] +(20) Project [codegen id : 7] Output [1]: [wr_order_number#14] Input [2]: [wr_order_number#14, wr_returned_date_sk#15] -(19) Exchange +(21) Exchange Input [1]: [wr_order_number#14] Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] -(20) Sort [codegen id : 8] +(22) Sort [codegen id : 8] Input [1]: [wr_order_number#14] Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 -(21) SortMergeJoin +(23) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] Right keys [1]: [wr_order_number#14] Join condition: None -(22) Scan parquet default.customer_address +(24) Scan parquet default.customer_address Output [2]: [ca_address_sk#17, ca_state#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 9] +(25) ColumnarToRow [codegen id : 9] Input [2]: [ca_address_sk#17, ca_state#18] -(24) Filter [codegen id : 9] +(26) Filter [codegen id : 9] Input [2]: [ca_address_sk#17, ca_state#18] Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = IL)) AND isnotnull(ca_address_sk#17)) -(25) Project [codegen id : 9] +(27) Project [codegen id : 9] Output [1]: [ca_address_sk#17] Input [2]: [ca_address_sk#17, ca_state#18] -(26) BroadcastExchange +(28) BroadcastExchange Input [1]: [ca_address_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(27) BroadcastHashJoin [codegen id : 12] +(29) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#17] Join condition: None -(28) Project [codegen id : 12] +(30) Project [codegen id : 12] Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#17] -(29) Scan parquet default.web_site +(31) Scan parquet default.web_site Output [2]: [web_site_sk#20, web_company_name#21] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 10] +(32) ColumnarToRow [codegen id : 10] Input [2]: [web_site_sk#20, web_company_name#21] -(31) Filter [codegen id : 10] +(33) Filter [codegen id : 10] Input [2]: [web_site_sk#20, web_company_name#21] -Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri)) AND isnotnull(web_site_sk#20)) +Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri )) AND isnotnull(web_site_sk#20)) -(32) Project [codegen id : 10] +(34) Project [codegen id : 10] Output [1]: [web_site_sk#20] Input [2]: [web_site_sk#20, web_company_name#21] -(33) BroadcastExchange +(35) BroadcastExchange Input [1]: [web_site_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(34) BroadcastHashJoin [codegen id : 12] +(36) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#20] Join condition: None -(35) Project [codegen id : 12] +(37) Project [codegen id : 12] Output [4]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#20] -(36) Scan parquet default.date_dim +(38) Scan parquet default.date_dim Output [2]: [d_date_sk#23, d_date#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 11] +(39) ColumnarToRow [codegen id : 11] Input [2]: [d_date_sk#23, d_date#24] -(38) Filter [codegen id : 11] +(40) Filter [codegen id : 11] Input [2]: [d_date_sk#23, d_date#24] -Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 10623)) AND (d_date#24 <= 10683)) AND isnotnull(d_date_sk#23)) +Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 1999-02-01)) AND (d_date#24 <= 1999-04-02)) AND isnotnull(d_date_sk#23)) -(39) Project [codegen id : 11] +(41) Project [codegen id : 11] Output [1]: [d_date_sk#23] Input [2]: [d_date_sk#23, d_date#24] -(40) BroadcastExchange +(42) BroadcastExchange Input [1]: [d_date_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] -(41) BroadcastHashJoin [codegen id : 12] +(43) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#23] Join condition: None -(42) Project [codegen id : 12] +(44) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#23] -(43) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(44) Exchange +(46) Exchange Input [3]: [ws_order_number#5, sum#28, sum#29] Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] -(45) HashAggregate [codegen id : 13] +(47) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(46) HashAggregate [codegen id : 13] +(48) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] Results [3]: [sum#28, sum#29, count#32] -(47) Exchange +(49) Exchange Input [3]: [sum#28, sum#29, count#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] -(48) HashAggregate [codegen id : 14] +(50) HashAggregate [codegen id : 14] Input [3]: [sum#28, sum#29, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] -(49) Sort [codegen id : 14] +(51) Sort [codegen id : 14] Input [3]: [order count #34, total shipping cost #35, total net profit #36] Arguments: [order count #34 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt index 3ef7977c4bc64..6a7459768a8fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt @@ -32,7 +32,7 @@ WholeStageCodegen (14) Exchange [ws_order_number] #4 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_warehouse_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] @@ -42,18 +42,20 @@ WholeStageCodegen (14) Exchange [ws_order_number] #5 WholeStageCodegen (3) Project [ws_warehouse_sk,ws_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + Filter [ws_order_number,ws_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] WholeStageCodegen (8) Sort [wr_order_number] InputAdapter Exchange [wr_order_number] #6 WholeStageCodegen (7) Project [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + Filter [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter BroadcastExchange #7 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index efa09c2f625cd..e6001eae60e57 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -1,60 +1,62 @@ == Physical Plan == -* Sort (49) -+- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- SortMergeJoin LeftAnti (21) - : : : :- * Sort (15) - : : : : +- Exchange (14) - : : : : +- * Project (13) - : : : : +- SortMergeJoin LeftSemi (12) +* Sort (51) ++- * HashAggregate (50) + +- Exchange (49) + +- * HashAggregate (48) + +- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- SortMergeJoin LeftAnti (23) + : : : :- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Project (14) + : : : : +- SortMergeJoin LeftSemi (13) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.web_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * Project (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.web_sales (7) - : : : +- * Sort (20) - : : : +- Exchange (19) - : : : +- * Project (18) - : : : +- * ColumnarToRow (17) - : : : +- Scan parquet default.web_returns (16) - : : +- BroadcastExchange (26) - : : +- * Project (25) - : : +- * Filter (24) - : : +- * ColumnarToRow (23) - : : +- Scan parquet default.date_dim (22) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- * ColumnarToRow (30) - : +- Scan parquet default.customer_address (29) - +- BroadcastExchange (40) - +- * Project (39) - +- * Filter (38) - +- * ColumnarToRow (37) - +- Scan parquet default.web_site (36) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.web_sales (7) + : : : +- * Sort (22) + : : : +- Exchange (21) + : : : +- * Project (20) + : : : +- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.web_returns (17) + : : +- BroadcastExchange (28) + : : +- * Project (27) + : : +- * Filter (26) + : : +- * ColumnarToRow (25) + : : +- Scan parquet default.date_dim (24) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.customer_address (31) + +- BroadcastExchange (42) + +- * Project (41) + +- * Filter (40) + +- * ColumnarToRow (39) + +- Scan parquet default.web_site (38) (1) Scan parquet default.web_sales Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -62,7 +64,7 @@ Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (3) Filter [codegen id : 1] Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) +Condition : ((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#5)) AND isnotnull(ws_warehouse_sk#4)) (4) Project [codegen id : 1] Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -80,196 +82,206 @@ Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -(9) Project [codegen id : 3] +(9) Filter [codegen id : 3] +Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Condition : (isnotnull(ws_order_number#5) AND isnotnull(ws_warehouse_sk#4)) + +(10) Project [codegen id : 3] Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -(10) Exchange +(11) Exchange Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] -(11) Sort [codegen id : 4] +(12) Sort [codegen id : 4] Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin +(13) SortMergeJoin Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#5#11] Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) -(13) Project [codegen id : 5] +(14) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(14) Exchange +(15) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] -(15) Sort [codegen id : 6] +(16) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 -(16) Scan parquet default.web_returns +(17) Scan parquet default.web_returns Output [2]: [wr_order_number#14, wr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 7] +(18) ColumnarToRow [codegen id : 7] +Input [2]: [wr_order_number#14, wr_returned_date_sk#15] + +(19) Filter [codegen id : 7] Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Condition : isnotnull(wr_order_number#14) -(18) Project [codegen id : 7] +(20) Project [codegen id : 7] Output [1]: [wr_order_number#14] Input [2]: [wr_order_number#14, wr_returned_date_sk#15] -(19) Exchange +(21) Exchange Input [1]: [wr_order_number#14] Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] -(20) Sort [codegen id : 8] +(22) Sort [codegen id : 8] Input [1]: [wr_order_number#14] Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 -(21) SortMergeJoin +(23) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] Right keys [1]: [wr_order_number#14] Join condition: None -(22) Scan parquet default.date_dim +(24) Scan parquet default.date_dim Output [2]: [d_date_sk#17, d_date#18] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 9] +(25) ColumnarToRow [codegen id : 9] Input [2]: [d_date_sk#17, d_date#18] -(24) Filter [codegen id : 9] +(26) Filter [codegen id : 9] Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 10623)) AND (d_date#18 <= 10683)) AND isnotnull(d_date_sk#17)) +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-01)) AND (d_date#18 <= 1999-04-02)) AND isnotnull(d_date_sk#17)) -(25) Project [codegen id : 9] +(27) Project [codegen id : 9] Output [1]: [d_date_sk#17] Input [2]: [d_date_sk#17, d_date#18] -(26) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(27) BroadcastHashJoin [codegen id : 12] +(29) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#17] Join condition: None -(28) Project [codegen id : 12] +(30) Project [codegen id : 12] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#17] -(29) Scan parquet default.customer_address +(31) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 10] +(32) ColumnarToRow [codegen id : 10] Input [2]: [ca_address_sk#20, ca_state#21] -(31) Filter [codegen id : 10] +(33) Filter [codegen id : 10] Input [2]: [ca_address_sk#20, ca_state#21] Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = IL)) AND isnotnull(ca_address_sk#20)) -(32) Project [codegen id : 10] +(34) Project [codegen id : 10] Output [1]: [ca_address_sk#20] Input [2]: [ca_address_sk#20, ca_state#21] -(33) BroadcastExchange +(35) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(34) BroadcastHashJoin [codegen id : 12] +(36) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#20] Join condition: None -(35) Project [codegen id : 12] +(37) Project [codegen id : 12] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#20] -(36) Scan parquet default.web_site +(38) Scan parquet default.web_site Output [2]: [web_site_sk#23, web_company_name#24] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 11] +(39) ColumnarToRow [codegen id : 11] Input [2]: [web_site_sk#23, web_company_name#24] -(38) Filter [codegen id : 11] +(40) Filter [codegen id : 11] Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri)) AND isnotnull(web_site_sk#23)) +Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri )) AND isnotnull(web_site_sk#23)) -(39) Project [codegen id : 11] +(41) Project [codegen id : 11] Output [1]: [web_site_sk#23] Input [2]: [web_site_sk#23, web_company_name#24] -(40) BroadcastExchange +(42) BroadcastExchange Input [1]: [web_site_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] -(41) BroadcastHashJoin [codegen id : 12] +(43) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#23] Join condition: None -(42) Project [codegen id : 12] +(44) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#23] -(43) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(44) Exchange +(46) Exchange Input [3]: [ws_order_number#5, sum#28, sum#29] Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] -(45) HashAggregate [codegen id : 13] +(47) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(46) HashAggregate [codegen id : 13] +(48) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] Results [3]: [sum#28, sum#29, count#32] -(47) Exchange +(49) Exchange Input [3]: [sum#28, sum#29, count#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] -(48) HashAggregate [codegen id : 14] +(50) HashAggregate [codegen id : 14] Input [3]: [sum#28, sum#29, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] -(49) Sort [codegen id : 14] +(51) Sort [codegen id : 14] Input [3]: [order count #34, total shipping cost #35, total net profit #36] Arguments: [order count #34 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 6bcd2aa24ccbb..1282ca0d0f702 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -32,7 +32,7 @@ WholeStageCodegen (14) Exchange [ws_order_number] #4 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_warehouse_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] @@ -42,18 +42,20 @@ WholeStageCodegen (14) Exchange [ws_order_number] #5 WholeStageCodegen (3) Project [ws_warehouse_sk,ws_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + Filter [ws_order_number,ws_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] WholeStageCodegen (8) Sort [wr_order_number] InputAdapter Exchange [wr_order_number] #6 WholeStageCodegen (7) Project [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + Filter [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter BroadcastExchange #7 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt index 725939ef24f1b..d111152b95384 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt @@ -69,7 +69,7 @@ Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -77,7 +77,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num (3) Filter [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) +Condition : (((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#4)) (4) Project [codegen id : 1] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] @@ -252,7 +252,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Output [2]: [web_site_sk#24, web_company_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 19] @@ -260,7 +260,7 @@ Input [2]: [web_site_sk#24, web_company_name#25] (46) Filter [codegen id : 19] Input [2]: [web_site_sk#24, web_company_name#25] -Condition : ((isnotnull(web_company_name#25) AND (web_company_name#25 = pri)) AND isnotnull(web_site_sk#24)) +Condition : ((isnotnull(web_company_name#25) AND (web_company_name#25 = pri )) AND isnotnull(web_site_sk#24)) (47) Project [codegen id : 19] Output [1]: [web_site_sk#24] @@ -291,7 +291,7 @@ Input [2]: [d_date_sk#27, d_date#28] (53) Filter [codegen id : 20] Input [2]: [d_date_sk#27, d_date#28] -Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 10623)) AND (d_date#28 <= 10683)) AND isnotnull(d_date_sk#27)) +Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 1999-02-01)) AND (d_date#28 <= 1999-04-02)) AND isnotnull(d_date_sk#27)) (54) Project [codegen id : 20] Output [1]: [d_date_sk#27] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt index 439f3935c65a3..53b0ccfc3e218 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt @@ -29,7 +29,7 @@ WholeStageCodegen (23) Exchange [ws_order_number] #4 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index 9f15375b5cfc0..f57897991b8fa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -69,7 +69,7 @@ Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -77,7 +77,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num (3) Filter [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) +Condition : (((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#4)) (4) Project [codegen id : 1] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] @@ -229,7 +229,7 @@ Input [2]: [d_date_sk#21, d_date#22] (39) Filter [codegen id : 18] Input [2]: [d_date_sk#21, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 10623)) AND (d_date#22 <= 10683)) AND isnotnull(d_date_sk#21)) +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-01)) AND (d_date#22 <= 1999-04-02)) AND isnotnull(d_date_sk#21)) (40) Project [codegen id : 18] Output [1]: [d_date_sk#21] @@ -283,7 +283,7 @@ Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_ Output [2]: [web_site_sk#27, web_company_name#28] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 20] @@ -291,7 +291,7 @@ Input [2]: [web_site_sk#27, web_company_name#28] (53) Filter [codegen id : 20] Input [2]: [web_site_sk#27, web_company_name#28] -Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri)) AND isnotnull(web_site_sk#27)) +Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri )) AND isnotnull(web_site_sk#27)) (54) Project [codegen id : 20] Output [1]: [web_site_sk#27] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index adf95b02f07e5..16bb4ac84f00c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -29,7 +29,7 @@ WholeStageCodegen (23) Exchange [ws_order_number] #4 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index 58d28b28dae10..422a89af349bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -1,60 +1,63 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * Project (45) - +- * BroadcastHashJoin Inner BuildLeft (44) - :- BroadcastExchange (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (32) - : : +- SortMergeJoin LeftSemi (31) - : : :- SortMergeJoin LeftSemi (17) +TakeOrderedAndProject (52) ++- * HashAggregate (51) + +- Exchange (50) + +- * HashAggregate (49) + +- * Project (48) + +- * BroadcastHashJoin Inner BuildLeft (47) + :- BroadcastExchange (43) + : +- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (35) + : : +- SortMergeJoin LeftSemi (34) + : : :- SortMergeJoin LeftSemi (18) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.customer (1) - : : : +- * Sort (16) - : : : +- Exchange (15) - : : : +- * Project (14) - : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : :- * ColumnarToRow (7) - : : : : +- Scan parquet default.store_sales (6) - : : : +- BroadcastExchange (12) - : : : +- * Project (11) - : : : +- * Filter (10) - : : : +- * ColumnarToRow (9) - : : : +- Scan parquet default.date_dim (8) - : : +- * Sort (30) - : : +- Exchange (29) - : : +- Union (28) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * ColumnarToRow (19) - : : : : +- Scan parquet default.web_sales (18) - : : : +- ReusedExchange (20) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * ColumnarToRow (24) - : : : +- Scan parquet default.catalog_sales (23) - : : +- ReusedExchange (25) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- * Filter (35) - : +- * ColumnarToRow (34) - : +- Scan parquet default.customer_address (33) - +- * Filter (43) - +- * ColumnarToRow (42) - +- Scan parquet default.customer_demographics (41) + : : : +- * Sort (17) + : : : +- Exchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.store_sales (6) + : : : +- BroadcastExchange (13) + : : : +- * Project (12) + : : : +- * Filter (11) + : : : +- * ColumnarToRow (10) + : : : +- Scan parquet default.date_dim (9) + : : +- * Sort (33) + : : +- Exchange (32) + : : +- Union (31) + : : :- * Project (24) + : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : :- * Filter (21) + : : : : +- * ColumnarToRow (20) + : : : : +- Scan parquet default.web_sales (19) + : : : +- ReusedExchange (22) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * Filter (27) + : : : +- * ColumnarToRow (26) + : : : +- Scan parquet default.catalog_sales (25) + : : +- ReusedExchange (28) + : +- BroadcastExchange (40) + : +- * Project (39) + : +- * Filter (38) + : +- * ColumnarToRow (37) + : +- Scan parquet default.customer_address (36) + +- * Filter (46) + +- * ColumnarToRow (45) + +- Scan parquet default.customer_demographics (44) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -62,7 +65,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -77,209 +80,224 @@ Output [2]: [ss_customer_sk#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -(8) Scan parquet default.date_dim +(8) Filter [codegen id : 4] +Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] +Condition : isnotnull(ss_customer_sk#5) + +(9) Scan parquet default.date_dim Output [3]: [d_date_sk#8, d_year#9, d_moy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(9) ColumnarToRow [codegen id : 3] +(10) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -(10) Filter [codegen id : 3] +(11) Filter [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] Condition : (((((isnotnull(d_year#9) AND isnotnull(d_moy#10)) AND (d_year#9 = 2002)) AND (d_moy#10 >= 4)) AND (d_moy#10 <= 7)) AND isnotnull(d_date_sk#8)) -(11) Project [codegen id : 3] +(12) Project [codegen id : 3] Output [1]: [d_date_sk#8] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -(12) BroadcastExchange +(13) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(13) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#8] Join condition: None -(14) Project [codegen id : 4] +(15) Project [codegen id : 4] Output [1]: [ss_customer_sk#5] Input [3]: [ss_customer_sk#5, ss_sold_date_sk#6, d_date_sk#8] -(15) Exchange +(16) Exchange Input [1]: [ss_customer_sk#5] Arguments: hashpartitioning(ss_customer_sk#5, 5), ENSURE_REQUIREMENTS, [id=#12] -(16) Sort [codegen id : 5] +(17) Sort [codegen id : 5] Input [1]: [ss_customer_sk#5] Arguments: [ss_customer_sk#5 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin +(18) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#5] Join condition: None -(18) Scan parquet default.web_sales +(19) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#14), dynamicpruningexpression(ws_sold_date_sk#14 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(20) ColumnarToRow [codegen id : 7] +Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] + +(21) Filter [codegen id : 7] Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] +Condition : isnotnull(ws_bill_customer_sk#13) -(20) ReusedExchange [Reuses operator id: 12] +(22) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -(21) BroadcastHashJoin [codegen id : 7] +(23) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] Right keys [1]: [d_date_sk#8] Join condition: None -(22) Project [codegen id : 7] +(24) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13 AS customer_sk#15] Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] -(23) Scan parquet default.catalog_sales +(25) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 9] +(26) ColumnarToRow [codegen id : 9] Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -(25) ReusedExchange [Reuses operator id: 12] +(27) Filter [codegen id : 9] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Condition : isnotnull(cs_ship_customer_sk#16) + +(28) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -(26) BroadcastHashJoin [codegen id : 9] +(29) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#8] Join condition: None -(27) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [1]: [cs_ship_customer_sk#16 AS customer_sk#18] Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] -(28) Union +(31) Union -(29) Exchange +(32) Exchange Input [1]: [customer_sk#15] Arguments: hashpartitioning(customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] -(30) Sort [codegen id : 10] +(33) Sort [codegen id : 10] Input [1]: [customer_sk#15] Arguments: [customer_sk#15 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#15] Join condition: None -(32) Project [codegen id : 12] +(35) Project [codegen id : 12] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(33) Scan parquet default.customer_address +(36) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 11] +(37) ColumnarToRow [codegen id : 11] Input [2]: [ca_address_sk#20, ca_county#21] -(35) Filter [codegen id : 11] +(38) Filter [codegen id : 11] Input [2]: [ca_address_sk#20, ca_county#21] Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) -(36) Project [codegen id : 11] +(39) Project [codegen id : 11] Output [1]: [ca_address_sk#20] Input [2]: [ca_address_sk#20, ca_county#21] -(37) BroadcastExchange +(40) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(38) BroadcastHashJoin [codegen id : 12] +(41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#20] Join condition: None -(39) Project [codegen id : 12] +(42) Project [codegen id : 12] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] -(40) BroadcastExchange +(43) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] -(41) Scan parquet default.customer_demographics +(44) Scan parquet default.customer_demographics Output [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(42) ColumnarToRow +(45) ColumnarToRow Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -(43) Filter +(46) Filter Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Condition : isnotnull(cd_demo_sk#24) -(44) BroadcastHashJoin [codegen id : 13] +(47) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#24] Join condition: None -(45) Project [codegen id : 13] +(48) Project [codegen id : 13] Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -(46) HashAggregate [codegen id : 13] +(49) HashAggregate [codegen id : 13] Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#33] Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] -(47) Exchange +(50) Exchange Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#35] -(48) HashAggregate [codegen id : 14] +(51) HashAggregate [codegen id : 14] Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#36] Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#36 AS cnt1#37, cd_purchase_estimate#28, count(1)#36 AS cnt2#38, cd_credit_rating#29, count(1)#36 AS cnt3#39, cd_dep_count#30, count(1)#36 AS cnt4#40, cd_dep_employed_count#31, count(1)#36 AS cnt5#41, cd_dep_college_count#32, count(1)#36 AS cnt6#42] -(49) TakeOrderedAndProject +(52) TakeOrderedAndProject Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -ReusedExchange (50) +ReusedExchange (53) -(50) ReusedExchange [Reuses operator id: 12] +(53) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt index 220059a6f5e27..1d4fd5a709520 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -32,11 +32,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #5 WholeStageCodegen (3) @@ -53,19 +54,21 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (7) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (9) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [cs_ship_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index eac6acc3543ca..1a258296802b5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -1,56 +1,59 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- Exchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * BroadcastHashJoin LeftSemi BuildRight (27) - : : :- * BroadcastHashJoin LeftSemi BuildRight (14) +TakeOrderedAndProject (48) ++- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (31) + : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : :- * BroadcastHashJoin LeftSemi BuildRight (15) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.customer (1) - : : : +- BroadcastExchange (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * ColumnarToRow (5) - : : : : +- Scan parquet default.store_sales (4) - : : : +- BroadcastExchange (10) - : : : +- * Project (9) - : : : +- * Filter (8) - : : : +- * ColumnarToRow (7) - : : : +- Scan parquet default.date_dim (6) - : : +- BroadcastExchange (26) - : : +- Union (25) - : : :- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * ColumnarToRow (16) - : : : : +- Scan parquet default.web_sales (15) - : : : +- ReusedExchange (17) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * ColumnarToRow (21) - : : : +- Scan parquet default.catalog_sales (20) - : : +- ReusedExchange (22) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- * ColumnarToRow (30) - : +- Scan parquet default.customer_address (29) - +- BroadcastExchange (39) - +- * Filter (38) - +- * ColumnarToRow (37) - +- Scan parquet default.customer_demographics (36) + : : : +- BroadcastExchange (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.store_sales (4) + : : : +- BroadcastExchange (11) + : : : +- * Project (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.date_dim (7) + : : +- BroadcastExchange (29) + : : +- Union (28) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Filter (18) + : : : : +- * ColumnarToRow (17) + : : : : +- Scan parquet default.web_sales (16) + : : : +- ReusedExchange (19) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Filter (24) + : : : +- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (25) + : +- BroadcastExchange (36) + : +- * Project (35) + : +- * Filter (34) + : +- * ColumnarToRow (33) + : +- Scan parquet default.customer_address (32) + +- BroadcastExchange (42) + +- * Filter (41) + +- * ColumnarToRow (40) + +- Scan parquet default.customer_demographics (39) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -58,208 +61,223 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) Scan parquet default.date_dim +(6) Filter [codegen id : 2] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Condition : isnotnull(ss_customer_sk#4) + +(7) Scan parquet default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 1] +(8) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -(8) Filter [codegen id : 1] +(9) Filter [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) -(9) Project [codegen id : 1] +(10) Project [codegen id : 1] Output [1]: [d_date_sk#7] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -(10) BroadcastExchange +(11) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] -(11) BroadcastHashJoin [codegen id : 2] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join condition: None -(12) Project [codegen id : 2] +(13) Project [codegen id : 2] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -(13) BroadcastExchange +(14) BroadcastExchange Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(14) BroadcastHashJoin [codegen id : 9] +(15) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join condition: None -(15) Scan parquet default.web_sales +(16) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 4] +(17) ColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] + +(18) Filter [codegen id : 4] Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +Condition : isnotnull(ws_bill_customer_sk#12) -(17) ReusedExchange [Reuses operator id: 10] +(19) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#7] -(18) BroadcastHashJoin [codegen id : 4] +(20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] Right keys [1]: [d_date_sk#7] Join condition: None -(19) Project [codegen id : 4] +(21) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12 AS customer_sk#14] Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] -(20) Scan parquet default.catalog_sales +(22) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(21) ColumnarToRow [codegen id : 6] +(23) ColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] + +(24) Filter [codegen id : 6] Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_ship_customer_sk#15) -(22) ReusedExchange [Reuses operator id: 10] +(25) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#7] -(23) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#7] Join condition: None -(24) Project [codegen id : 6] +(27) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#15 AS customer_sk#17] Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] -(25) Union +(28) Union -(26) BroadcastExchange +(29) BroadcastExchange Input [1]: [customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(27) BroadcastHashJoin [codegen id : 9] +(30) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#14] Join condition: None -(28) Project [codegen id : 9] +(31) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(29) Scan parquet default.customer_address +(32) Scan parquet default.customer_address Output [2]: [ca_address_sk#19, ca_county#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 7] +(33) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#19, ca_county#20] -(31) Filter [codegen id : 7] +(34) Filter [codegen id : 7] Input [2]: [ca_address_sk#19, ca_county#20] Condition : (ca_county#20 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#19)) -(32) Project [codegen id : 7] +(35) Project [codegen id : 7] Output [1]: [ca_address_sk#19] Input [2]: [ca_address_sk#19, ca_county#20] -(33) BroadcastExchange +(36) BroadcastExchange Input [1]: [ca_address_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(34) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#19] Join condition: None -(35) Project [codegen id : 9] +(38) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] -(36) Scan parquet default.customer_demographics +(39) Scan parquet default.customer_demographics Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 8] +(40) ColumnarToRow [codegen id : 8] Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(38) Filter [codegen id : 8] +(41) Filter [codegen id : 8] Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Condition : isnotnull(cd_demo_sk#22) -(39) BroadcastExchange +(42) BroadcastExchange Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] -(40) BroadcastHashJoin [codegen id : 9] +(43) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#22] Join condition: None -(41) Project [codegen id : 9] +(44) Project [codegen id : 9] Output [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(42) HashAggregate [codegen id : 9] +(45) HashAggregate [codegen id : 9] Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#32] Results [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] -(43) Exchange +(46) Exchange Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#34] -(44) HashAggregate [codegen id : 10] +(47) HashAggregate [codegen id : 10] Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#35] Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#35 AS cnt1#36, cd_purchase_estimate#26, count(1)#35 AS cnt2#37, cd_credit_rating#27, count(1)#35 AS cnt3#38, cd_dep_count#28, count(1)#35 AS cnt4#39, cd_dep_employed_count#29, count(1)#35 AS cnt5#40, cd_dep_college_count#30, count(1)#35 AS cnt6#41] -(45) TakeOrderedAndProject +(48) TakeOrderedAndProject Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (46) +ReusedExchange (49) -(46) ReusedExchange [Reuses operator id: 10] +(49) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#7] -Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 5077b91e30c1b..72526cd0b007a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -21,11 +21,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -40,19 +41,21 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [cs_ship_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter 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 312aba94bd9c6..f100c3e7e6ebf 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 @@ -144,7 +144,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 20] @@ -152,7 +152,7 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +Condition : (((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_item_sk#7)) (9) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] 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 1d5907e319658..be55d3b2fecfc 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 @@ -75,7 +75,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (20) 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] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 258224e574cfb..b4ab1eb60e90f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -120,7 +120,7 @@ Condition : isnotnull(ss_item_sk#1) 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)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -128,7 +128,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] 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)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (7) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 3507d76bcf9b9..592d91ddf08fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -67,7 +67,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] 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 9b47400d3b780..b8f60c510269a 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 @@ -241,7 +241,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 20] @@ -249,7 +249,7 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +Condition : (((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_item_sk#7)) (9) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] 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 d27eb5a32b387..d418fe48a4d67 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 @@ -97,7 +97,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (20) 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] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index 04fe14cab2c9b..c7b4a326a053f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -214,7 +214,7 @@ Condition : isnotnull(ss_item_sk#1) 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)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -222,7 +222,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] 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)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (7) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 1634c1e247a12..80c58cd303dc5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -89,7 +89,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt index 0a861f4ce8e6d..9657cdfd07984 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt @@ -1,68 +1,69 @@ == Physical Plan == -TakeOrderedAndProject (57) -+- * HashAggregate (56) - +- Exchange (55) - +- * HashAggregate (54) - +- * Project (53) - +- * SortMergeJoin Inner (52) - :- * Sort (46) - : +- Exchange (45) - : +- * Project (44) - : +- * SortMergeJoin Inner (43) - : :- * Sort (37) - : : +- Exchange (36) - : : +- * Project (35) - : : +- * Filter (34) - : : +- SortMergeJoin ExistenceJoin(exists#1) (33) - : : :- SortMergeJoin ExistenceJoin(exists#2) (25) - : : : :- SortMergeJoin LeftSemi (17) +TakeOrderedAndProject (58) ++- * HashAggregate (57) + +- Exchange (56) + +- * HashAggregate (55) + +- * Project (54) + +- * SortMergeJoin Inner (53) + :- * Sort (47) + : +- Exchange (46) + : +- * Project (45) + : +- * SortMergeJoin Inner (44) + : :- * Sort (38) + : : +- Exchange (37) + : : +- * Project (36) + : : +- * Filter (35) + : : +- SortMergeJoin ExistenceJoin(exists#1) (34) + : : :- SortMergeJoin ExistenceJoin(exists#2) (26) + : : : :- SortMergeJoin LeftSemi (18) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- * Sort (16) - : : : : +- Exchange (15) - : : : : +- * Project (14) - : : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : : :- * ColumnarToRow (7) - : : : : : +- Scan parquet default.store_sales (6) - : : : : +- BroadcastExchange (12) - : : : : +- * Project (11) - : : : : +- * Filter (10) - : : : : +- * ColumnarToRow (9) - : : : : +- Scan parquet default.date_dim (8) - : : : +- * Sort (24) - : : : +- Exchange (23) - : : : +- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * ColumnarToRow (19) - : : : : +- Scan parquet default.web_sales (18) - : : : +- ReusedExchange (20) - : : +- * Sort (32) - : : +- Exchange (31) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * ColumnarToRow (27) - : : : +- Scan parquet default.catalog_sales (26) - : : +- ReusedExchange (28) - : +- * Sort (42) - : +- Exchange (41) - : +- * Filter (40) - : +- * ColumnarToRow (39) - : +- Scan parquet default.customer_address (38) - +- * Sort (51) - +- Exchange (50) - +- * Filter (49) - +- * ColumnarToRow (48) - +- Scan parquet default.customer_demographics (47) + : : : : +- * Sort (17) + : : : : +- Exchange (16) + : : : : +- * Project (15) + : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * Filter (11) + : : : : +- * ColumnarToRow (10) + : : : : +- Scan parquet default.date_dim (9) + : : : +- * Sort (25) + : : : +- Exchange (24) + : : : +- * Project (23) + : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : :- * ColumnarToRow (20) + : : : : +- Scan parquet default.web_sales (19) + : : : +- ReusedExchange (21) + : : +- * Sort (33) + : : +- Exchange (32) + : : +- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * ColumnarToRow (28) + : : : +- Scan parquet default.catalog_sales (27) + : : +- ReusedExchange (29) + : +- * Sort (43) + : +- Exchange (42) + : +- * Filter (41) + : +- * ColumnarToRow (40) + : +- Scan parquet default.customer_address (39) + +- * Sort (52) + +- Exchange (51) + +- * Filter (50) + +- * ColumnarToRow (49) + +- Scan parquet default.customer_demographics (48) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -70,7 +71,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -85,244 +86,249 @@ Output [2]: [ss_customer_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -(8) Scan parquet default.date_dim +(8) Filter [codegen id : 4] +Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] +Condition : isnotnull(ss_customer_sk#7) + +(9) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_qoy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(9) ColumnarToRow [codegen id : 3] +(10) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] -(10) Filter [codegen id : 3] +(11) Filter [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_qoy#12)) AND (d_year#11 = 2002)) AND (d_qoy#12 < 4)) AND isnotnull(d_date_sk#10)) -(11) Project [codegen id : 3] +(12) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] -(12) BroadcastExchange +(13) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(13) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(14) Project [codegen id : 4] +(15) Project [codegen id : 4] Output [1]: [ss_customer_sk#7] Input [3]: [ss_customer_sk#7, ss_sold_date_sk#8, d_date_sk#10] -(15) Exchange +(16) Exchange Input [1]: [ss_customer_sk#7] Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] -(16) Sort [codegen id : 5] +(17) Sort [codegen id : 5] Input [1]: [ss_customer_sk#7] Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin +(18) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#7] Join condition: None -(18) Scan parquet default.web_sales +(19) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#16), dynamicpruningexpression(ws_sold_date_sk#16 IN dynamicpruning#9)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(20) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] -(20) ReusedExchange [Reuses operator id: 12] +(21) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#10] -(21) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(22) Project [codegen id : 7] +(23) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] -(23) Exchange +(24) Exchange Input [1]: [ws_bill_customer_sk#15] Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] -(24) Sort [codegen id : 8] +(25) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] Arguments: [ws_bill_customer_sk#15 ASC NULLS FIRST], false, 0 -(25) SortMergeJoin +(26) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#15] Join condition: None -(26) Scan parquet default.catalog_sales +(27) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 10] +(28) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] -(28) ReusedExchange [Reuses operator id: 12] +(29) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#10] -(29) BroadcastHashJoin [codegen id : 10] +(30) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#19] Right keys [1]: [d_date_sk#10] Join condition: None -(30) Project [codegen id : 10] +(31) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#18] Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] -(31) Exchange +(32) Exchange Input [1]: [cs_ship_customer_sk#18] Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] -(32) Sort [codegen id : 11] +(33) Sort [codegen id : 11] Input [1]: [cs_ship_customer_sk#18] Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 -(33) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#18] Join condition: None -(34) Filter [codegen id : 12] +(35) Filter [codegen id : 12] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(35) Project [codegen id : 12] +(36) Project [codegen id : 12] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(36) Exchange +(37) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] -(37) Sort [codegen id : 13] +(38) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.customer_address +(39) Scan parquet default.customer_address Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 14] +(40) ColumnarToRow [codegen id : 14] Input [2]: [ca_address_sk#22, ca_state#23] -(40) Filter [codegen id : 14] +(41) Filter [codegen id : 14] Input [2]: [ca_address_sk#22, ca_state#23] Condition : isnotnull(ca_address_sk#22) -(41) Exchange +(42) Exchange Input [2]: [ca_address_sk#22, ca_state#23] Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] -(42) Sort [codegen id : 15] +(43) Sort [codegen id : 15] Input [2]: [ca_address_sk#22, ca_state#23] Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 -(43) SortMergeJoin [codegen id : 16] +(44) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#22] Join condition: None -(44) Project [codegen id : 16] +(45) Project [codegen id : 16] Output [2]: [c_current_cdemo_sk#4, ca_state#23] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] -(45) Exchange +(46) Exchange Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] -(46) Sort [codegen id : 17] +(47) Sort [codegen id : 17] Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(47) Scan parquet default.customer_demographics +(48) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 18] +(49) ColumnarToRow [codegen id : 18] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(49) Filter [codegen id : 18] +(50) Filter [codegen id : 18] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Condition : isnotnull(cd_demo_sk#26) -(50) Exchange +(51) Exchange Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] -(51) Sort [codegen id : 19] +(52) Sort [codegen id : 19] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 -(52) SortMergeJoin [codegen id : 20] +(53) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#26] Join condition: None -(53) Project [codegen id : 20] +(54) Project [codegen id : 20] Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(54) HashAggregate [codegen id : 20] +(55) HashAggregate [codegen id : 20] Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#29), partial_max(cd_dep_count#29), partial_sum(cd_dep_count#29), partial_avg(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_sum(cd_dep_employed_count#30), partial_avg(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_sum(cd_dep_college_count#31)] Aggregate Attributes [13]: [count#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45] Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] -(55) Exchange +(56) Exchange Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] -(56) HashAggregate [codegen id : 21] +(57) HashAggregate [codegen id : 21] Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [10]: [count(1), avg(cd_dep_count#29), max(cd_dep_count#29), sum(cd_dep_count#29), avg(cd_dep_employed_count#30), max(cd_dep_employed_count#30), sum(cd_dep_employed_count#30), avg(cd_dep_college_count#31), max(cd_dep_college_count#31), sum(cd_dep_college_count#31)] Aggregate Attributes [10]: [count(1)#60, avg(cd_dep_count#29)#61, max(cd_dep_count#29)#62, sum(cd_dep_count#29)#63, avg(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, sum(cd_dep_employed_count#30)#66, avg(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, sum(cd_dep_college_count#31)#69] Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, count(1)#60 AS cnt1#70, avg(cd_dep_count#29)#61 AS avg(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, sum(cd_dep_count#29)#63 AS sum(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, avg(cd_dep_employed_count#30)#64 AS avg(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, sum(cd_dep_employed_count#30)#66 AS sum(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, avg(cd_dep_college_count#31)#67 AS avg(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, sum(cd_dep_college_count#31)#69 AS sum(cd_dep_college_count)#81] -(57) TakeOrderedAndProject +(58) TakeOrderedAndProject Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (58) +ReusedExchange (59) -(58) ReusedExchange [Reuses operator id: 12] +(59) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt index ee441a8332a2a..7be34612be79f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -43,11 +43,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #6 + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #6 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index 12c8230c7e3bc..c803c1989723d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -1,57 +1,58 @@ == Physical Plan == -TakeOrderedAndProject (46) -+- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (30) - : : +- * Filter (29) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (28) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (21) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (14) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (31) + : : +- * Filter (30) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (29) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (22) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (13) - : : : : +- * Project (12) - : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (10) - : : : : +- * Project (9) - : : : : +- * Filter (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.date_dim (6) - : : : +- BroadcastExchange (20) - : : : +- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * ColumnarToRow (16) - : : : : +- Scan parquet default.web_sales (15) - : : : +- ReusedExchange (17) - : : +- BroadcastExchange (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (24) - : +- BroadcastExchange (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.customer_address (31) - +- BroadcastExchange (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet default.customer_demographics (37) + : : : : +- BroadcastExchange (14) + : : : : +- * Project (13) + : : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.store_sales (4) + : : : : +- BroadcastExchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.date_dim (7) + : : : +- BroadcastExchange (21) + : : : +- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * ColumnarToRow (17) + : : : : +- Scan parquet default.web_sales (16) + : : : +- ReusedExchange (18) + : : +- BroadcastExchange (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * ColumnarToRow (24) + : : : +- Scan parquet default.catalog_sales (23) + : : +- ReusedExchange (25) + : +- BroadcastExchange (35) + : +- * Filter (34) + : +- * ColumnarToRow (33) + : +- Scan parquet default.customer_address (32) + +- BroadcastExchange (41) + +- * Filter (40) + +- * ColumnarToRow (39) + +- Scan parquet default.customer_demographics (38) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -59,215 +60,220 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) Scan parquet default.date_dim +(6) Filter [codegen id : 2] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#6) + +(7) Scan parquet default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 1] +(8) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -(8) Filter [codegen id : 1] +(9) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(9) Project [codegen id : 1] +(10) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -(10) BroadcastExchange +(11) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] -(11) BroadcastHashJoin [codegen id : 2] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join condition: None -(12) Project [codegen id : 2] +(13) Project [codegen id : 2] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(13) BroadcastExchange +(14) BroadcastExchange Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(14) BroadcastHashJoin [codegen id : 9] +(15) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join condition: None -(15) Scan parquet default.web_sales +(16) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#15), dynamicpruningexpression(ws_sold_date_sk#15 IN dynamicpruning#8)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 4] +(17) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] -(17) ReusedExchange [Reuses operator id: 10] +(18) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#9] -(18) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] Right keys [1]: [d_date_sk#9] Join condition: None -(19) Project [codegen id : 4] +(20) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] -(20) BroadcastExchange +(21) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(21) BroadcastHashJoin [codegen id : 9] +(22) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#14] Join condition: None -(22) Scan parquet default.catalog_sales +(23) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 6] +(24) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] -(24) ReusedExchange [Reuses operator id: 10] +(25) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#9] -(25) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#9] Join condition: None -(26) Project [codegen id : 6] +(27) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#17] Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] -(27) BroadcastExchange +(28) BroadcastExchange Input [1]: [cs_ship_customer_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(28) BroadcastHashJoin [codegen id : 9] +(29) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#17] Join condition: None -(29) Filter [codegen id : 9] +(30) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(30) Project [codegen id : 9] +(31) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(31) Scan parquet default.customer_address +(32) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 7] +(33) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#20, ca_state#21] -(33) Filter [codegen id : 7] +(34) Filter [codegen id : 7] Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(34) BroadcastExchange +(35) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(35) BroadcastHashJoin [codegen id : 9] +(36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join condition: None -(36) Project [codegen id : 9] +(37) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, ca_state#21] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] -(37) Scan parquet default.customer_demographics +(38) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(38) ColumnarToRow [codegen id : 8] +(39) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(39) Filter [codegen id : 8] +(40) Filter [codegen id : 8] Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Condition : isnotnull(cd_demo_sk#23) -(40) BroadcastExchange +(41) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(41) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join condition: None -(42) Project [codegen id : 9] +(43) Project [codegen id : 9] Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) HashAggregate [codegen id : 9] +(44) HashAggregate [codegen id : 9] Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] Aggregate Attributes [13]: [count#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42] Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] -(44) Exchange +(45) Exchange Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] -(45) HashAggregate [codegen id : 10] +(46) HashAggregate [codegen id : 10] Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] -(46) TakeOrderedAndProject +(47) TakeOrderedAndProject Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -ReusedExchange (47) +ReusedExchange (48) -(47) ReusedExchange [Reuses operator id: 10] +(48) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#9] -Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index c6c9e1da58c26..c2dd037ca1b47 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -23,11 +23,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt index 1c1b8ad01abfa..bae142df5e6d6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt @@ -1,65 +1,68 @@ == Physical Plan == -TakeOrderedAndProject (54) -+- * HashAggregate (53) - +- Exchange (52) - +- * HashAggregate (51) - +- * Project (50) - +- * SortMergeJoin Inner (49) - :- * Sort (43) - : +- Exchange (42) - : +- * Project (41) - : +- * SortMergeJoin Inner (40) - : :- * Sort (34) - : : +- Exchange (33) - : : +- * Project (32) - : : +- SortMergeJoin LeftSemi (31) - : : :- SortMergeJoin LeftSemi (17) +TakeOrderedAndProject (57) ++- * HashAggregate (56) + +- Exchange (55) + +- * HashAggregate (54) + +- * Project (53) + +- * SortMergeJoin Inner (52) + :- * Sort (46) + : +- Exchange (45) + : +- * Project (44) + : +- * SortMergeJoin Inner (43) + : :- * Sort (37) + : : +- Exchange (36) + : : +- * Project (35) + : : +- SortMergeJoin LeftSemi (34) + : : :- SortMergeJoin LeftSemi (18) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.customer (1) - : : : +- * Sort (16) - : : : +- Exchange (15) - : : : +- * Project (14) - : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : :- * ColumnarToRow (7) - : : : : +- Scan parquet default.store_sales (6) - : : : +- BroadcastExchange (12) - : : : +- * Project (11) - : : : +- * Filter (10) - : : : +- * ColumnarToRow (9) - : : : +- Scan parquet default.date_dim (8) - : : +- * Sort (30) - : : +- Exchange (29) - : : +- Union (28) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * ColumnarToRow (19) - : : : : +- Scan parquet default.web_sales (18) - : : : +- ReusedExchange (20) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * ColumnarToRow (24) - : : : +- Scan parquet default.catalog_sales (23) - : : +- ReusedExchange (25) - : +- * Sort (39) - : +- Exchange (38) - : +- * Filter (37) - : +- * ColumnarToRow (36) - : +- Scan parquet default.customer_address (35) - +- * Sort (48) - +- Exchange (47) - +- * Filter (46) - +- * ColumnarToRow (45) - +- Scan parquet default.customer_demographics (44) + : : : +- * Sort (17) + : : : +- Exchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.store_sales (6) + : : : +- BroadcastExchange (13) + : : : +- * Project (12) + : : : +- * Filter (11) + : : : +- * ColumnarToRow (10) + : : : +- Scan parquet default.date_dim (9) + : : +- * Sort (33) + : : +- Exchange (32) + : : +- Union (31) + : : :- * Project (24) + : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : :- * Filter (21) + : : : : +- * ColumnarToRow (20) + : : : : +- Scan parquet default.web_sales (19) + : : : +- ReusedExchange (22) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * Filter (27) + : : : +- * ColumnarToRow (26) + : : : +- Scan parquet default.catalog_sales (25) + : : +- ReusedExchange (28) + : +- * Sort (42) + : +- Exchange (41) + : +- * Filter (40) + : +- * ColumnarToRow (39) + : +- Scan parquet default.customer_address (38) + +- * Sort (51) + +- Exchange (50) + +- * Filter (49) + +- * ColumnarToRow (48) + +- Scan parquet default.customer_demographics (47) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -67,7 +70,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -82,229 +85,244 @@ Output [2]: [ss_customer_sk#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -(8) Scan parquet default.date_dim +(8) Filter [codegen id : 4] +Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] +Condition : isnotnull(ss_customer_sk#5) + +(9) Scan parquet default.date_dim Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(9) ColumnarToRow [codegen id : 3] +(10) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -(10) Filter [codegen id : 3] +(11) Filter [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] Condition : ((((isnotnull(d_year#9) AND isnotnull(d_qoy#10)) AND (d_year#9 = 1999)) AND (d_qoy#10 < 4)) AND isnotnull(d_date_sk#8)) -(11) Project [codegen id : 3] +(12) Project [codegen id : 3] Output [1]: [d_date_sk#8] Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -(12) BroadcastExchange +(13) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(13) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#8] Join condition: None -(14) Project [codegen id : 4] +(15) Project [codegen id : 4] Output [1]: [ss_customer_sk#5] Input [3]: [ss_customer_sk#5, ss_sold_date_sk#6, d_date_sk#8] -(15) Exchange +(16) Exchange Input [1]: [ss_customer_sk#5] Arguments: hashpartitioning(ss_customer_sk#5, 5), ENSURE_REQUIREMENTS, [id=#12] -(16) Sort [codegen id : 5] +(17) Sort [codegen id : 5] Input [1]: [ss_customer_sk#5] Arguments: [ss_customer_sk#5 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin +(18) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#5] Join condition: None -(18) Scan parquet default.web_sales +(19) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#14), dynamicpruningexpression(ws_sold_date_sk#14 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(20) ColumnarToRow [codegen id : 7] +Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] + +(21) Filter [codegen id : 7] Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] +Condition : isnotnull(ws_bill_customer_sk#13) -(20) ReusedExchange [Reuses operator id: 12] +(22) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -(21) BroadcastHashJoin [codegen id : 7] +(23) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] Right keys [1]: [d_date_sk#8] Join condition: None -(22) Project [codegen id : 7] +(24) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13 AS customsk#15] Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] -(23) Scan parquet default.catalog_sales +(25) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 9] +(26) ColumnarToRow [codegen id : 9] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] + +(27) Filter [codegen id : 9] Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Condition : isnotnull(cs_ship_customer_sk#16) -(25) ReusedExchange [Reuses operator id: 12] +(28) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -(26) BroadcastHashJoin [codegen id : 9] +(29) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#8] Join condition: None -(27) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [1]: [cs_ship_customer_sk#16 AS customsk#18] Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] -(28) Union +(31) Union -(29) Exchange +(32) Exchange Input [1]: [customsk#15] Arguments: hashpartitioning(customsk#15, 5), ENSURE_REQUIREMENTS, [id=#19] -(30) Sort [codegen id : 10] +(33) Sort [codegen id : 10] Input [1]: [customsk#15] Arguments: [customsk#15 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [customsk#15] Join condition: None -(32) Project [codegen id : 11] +(35) Project [codegen id : 11] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(33) Exchange +(36) Exchange Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#20] -(34) Sort [codegen id : 12] +(37) Sort [codegen id : 12] Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: [c_current_addr_sk#3 ASC NULLS FIRST], false, 0 -(35) Scan parquet default.customer_address +(38) Scan parquet default.customer_address Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 13] +(39) ColumnarToRow [codegen id : 13] Input [2]: [ca_address_sk#21, ca_state#22] -(37) Filter [codegen id : 13] +(40) Filter [codegen id : 13] Input [2]: [ca_address_sk#21, ca_state#22] Condition : isnotnull(ca_address_sk#21) -(38) Exchange +(41) Exchange Input [2]: [ca_address_sk#21, ca_state#22] Arguments: hashpartitioning(ca_address_sk#21, 5), ENSURE_REQUIREMENTS, [id=#23] -(39) Sort [codegen id : 14] +(42) Sort [codegen id : 14] Input [2]: [ca_address_sk#21, ca_state#22] Arguments: [ca_address_sk#21 ASC NULLS FIRST], false, 0 -(40) SortMergeJoin [codegen id : 15] +(43) SortMergeJoin [codegen id : 15] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#21] Join condition: None -(41) Project [codegen id : 15] +(44) Project [codegen id : 15] Output [2]: [c_current_cdemo_sk#2, ca_state#22] Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21, ca_state#22] -(42) Exchange +(45) Exchange Input [2]: [c_current_cdemo_sk#2, ca_state#22] Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] -(43) Sort [codegen id : 16] +(46) Sort [codegen id : 16] Input [2]: [c_current_cdemo_sk#2, ca_state#22] Arguments: [c_current_cdemo_sk#2 ASC NULLS FIRST], false, 0 -(44) Scan parquet default.customer_demographics +(47) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 17] +(48) ColumnarToRow [codegen id : 17] Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(46) Filter [codegen id : 17] +(49) Filter [codegen id : 17] Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Condition : isnotnull(cd_demo_sk#25) -(47) Exchange +(50) Exchange Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] -(48) Sort [codegen id : 18] +(51) Sort [codegen id : 18] Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 -(49) SortMergeJoin [codegen id : 19] +(52) SortMergeJoin [codegen id : 19] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#25] Join condition: None -(50) Project [codegen id : 19] +(53) Project [codegen id : 19] Output [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(51) HashAggregate [codegen id : 19] +(54) HashAggregate [codegen id : 19] Input [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#28), partial_max(cd_dep_count#28), partial_sum(cd_dep_count#28), partial_avg(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_sum(cd_dep_employed_count#29), partial_avg(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_sum(cd_dep_college_count#30)] Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] Results [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -(52) Exchange +(55) Exchange Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] Arguments: hashpartitioning(ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] -(53) HashAggregate [codegen id : 20] +(56) HashAggregate [codegen id : 20] Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [10]: [count(1), avg(cd_dep_count#28), max(cd_dep_count#28), sum(cd_dep_count#28), avg(cd_dep_employed_count#29), max(cd_dep_employed_count#29), sum(cd_dep_employed_count#29), avg(cd_dep_college_count#30), max(cd_dep_college_count#30), sum(cd_dep_college_count#30)] Aggregate Attributes [10]: [count(1)#59, avg(cd_dep_count#28)#60, max(cd_dep_count#28)#61, sum(cd_dep_count#28)#62, avg(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, sum(cd_dep_employed_count#29)#65, avg(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, sum(cd_dep_college_count#30)#68] Results [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, count(1)#59 AS cnt1#69, avg(cd_dep_count#28)#60 AS avg(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, sum(cd_dep_count#28)#62 AS sum(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, avg(cd_dep_employed_count#29)#63 AS avg(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, sum(cd_dep_employed_count#29)#65 AS sum(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, avg(cd_dep_college_count#30)#66 AS avg(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, sum(cd_dep_college_count#30)#68 AS sum(cd_dep_college_count)#80] -(54) TakeOrderedAndProject +(57) TakeOrderedAndProject Input [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -ReusedExchange (55) +ReusedExchange (58) -(55) ReusedExchange [Reuses operator id: 12] +(58) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt index 9a090e0da62c8..419706064ea82 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt @@ -30,7 +30,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -41,11 +41,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #6 + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #6 WholeStageCodegen (3) @@ -62,19 +63,21 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (7) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #6 WholeStageCodegen (9) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [cs_ship_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index ffb07d398de71..12f35bdc6bb56 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -1,55 +1,58 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (28) - : : +- * BroadcastHashJoin LeftSemi BuildRight (27) - : : :- * BroadcastHashJoin LeftSemi BuildRight (14) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (31) + : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : :- * BroadcastHashJoin LeftSemi BuildRight (15) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.customer (1) - : : : +- BroadcastExchange (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * ColumnarToRow (5) - : : : : +- Scan parquet default.store_sales (4) - : : : +- BroadcastExchange (10) - : : : +- * Project (9) - : : : +- * Filter (8) - : : : +- * ColumnarToRow (7) - : : : +- Scan parquet default.date_dim (6) - : : +- BroadcastExchange (26) - : : +- Union (25) - : : :- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * ColumnarToRow (16) - : : : : +- Scan parquet default.web_sales (15) - : : : +- ReusedExchange (17) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * ColumnarToRow (21) - : : : +- Scan parquet default.catalog_sales (20) - : : +- ReusedExchange (22) - : +- BroadcastExchange (32) - : +- * Filter (31) - : +- * ColumnarToRow (30) - : +- Scan parquet default.customer_address (29) - +- BroadcastExchange (38) - +- * Filter (37) - +- * ColumnarToRow (36) - +- Scan parquet default.customer_demographics (35) + : : : +- BroadcastExchange (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.store_sales (4) + : : : +- BroadcastExchange (11) + : : : +- * Project (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.date_dim (7) + : : +- BroadcastExchange (29) + : : +- Union (28) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Filter (18) + : : : : +- * ColumnarToRow (17) + : : : : +- Scan parquet default.web_sales (16) + : : : +- ReusedExchange (19) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Filter (24) + : : : +- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (25) + : +- BroadcastExchange (35) + : +- * Filter (34) + : +- * ColumnarToRow (33) + : +- Scan parquet default.customer_address (32) + +- BroadcastExchange (41) + +- * Filter (40) + +- * ColumnarToRow (39) + +- Scan parquet default.customer_demographics (38) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -57,204 +60,219 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) Scan parquet default.date_dim +(6) Filter [codegen id : 2] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Condition : isnotnull(ss_customer_sk#4) + +(7) Scan parquet default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 1] +(8) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -(8) Filter [codegen id : 1] +(9) Filter [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) -(9) Project [codegen id : 1] +(10) Project [codegen id : 1] Output [1]: [d_date_sk#7] Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -(10) BroadcastExchange +(11) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] -(11) BroadcastHashJoin [codegen id : 2] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join condition: None -(12) Project [codegen id : 2] +(13) Project [codegen id : 2] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -(13) BroadcastExchange +(14) BroadcastExchange Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(14) BroadcastHashJoin [codegen id : 9] +(15) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join condition: None -(15) Scan parquet default.web_sales +(16) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 4] +(17) ColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] + +(18) Filter [codegen id : 4] Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +Condition : isnotnull(ws_bill_customer_sk#12) -(17) ReusedExchange [Reuses operator id: 10] +(19) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#7] -(18) BroadcastHashJoin [codegen id : 4] +(20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] Right keys [1]: [d_date_sk#7] Join condition: None -(19) Project [codegen id : 4] +(21) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12 AS customsk#14] Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] -(20) Scan parquet default.catalog_sales +(22) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(21) ColumnarToRow [codegen id : 6] +(23) ColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] + +(24) Filter [codegen id : 6] Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_ship_customer_sk#15) -(22) ReusedExchange [Reuses operator id: 10] +(25) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#7] -(23) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#7] Join condition: None -(24) Project [codegen id : 6] +(27) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#15 AS customsk#17] Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] -(25) Union +(28) Union -(26) BroadcastExchange +(29) BroadcastExchange Input [1]: [customsk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(27) BroadcastHashJoin [codegen id : 9] +(30) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customsk#14] Join condition: None -(28) Project [codegen id : 9] +(31) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(29) Scan parquet default.customer_address +(32) Scan parquet default.customer_address Output [2]: [ca_address_sk#19, ca_state#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 7] +(33) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#19, ca_state#20] -(31) Filter [codegen id : 7] +(34) Filter [codegen id : 7] Input [2]: [ca_address_sk#19, ca_state#20] Condition : isnotnull(ca_address_sk#19) -(32) BroadcastExchange +(35) BroadcastExchange Input [2]: [ca_address_sk#19, ca_state#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] -(33) BroadcastHashJoin [codegen id : 9] +(36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#19] Join condition: None -(34) Project [codegen id : 9] +(37) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, ca_state#20] Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19, ca_state#20] -(35) Scan parquet default.customer_demographics +(38) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 8] +(39) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(37) Filter [codegen id : 8] +(40) Filter [codegen id : 8] Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Condition : isnotnull(cd_demo_sk#22) -(38) BroadcastExchange +(41) BroadcastExchange Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] -(39) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#22] Join condition: None -(40) Project [codegen id : 9] +(43) Project [codegen id : 9] Output [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Input [8]: [c_current_cdemo_sk#2, ca_state#20, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(41) HashAggregate [codegen id : 9] +(44) HashAggregate [codegen id : 9] Input [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] Results [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -(42) Exchange +(45) Exchange Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] Arguments: hashpartitioning(ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [id=#55] -(43) HashAggregate [codegen id : 10] +(46) HashAggregate [codegen id : 10] Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] Aggregate Attributes [10]: [count(1)#56, avg(cd_dep_count#25)#57, max(cd_dep_count#25)#58, sum(cd_dep_count#25)#59, avg(cd_dep_employed_count#26)#60, max(cd_dep_employed_count#26)#61, sum(cd_dep_employed_count#26)#62, avg(cd_dep_college_count#27)#63, max(cd_dep_college_count#27)#64, sum(cd_dep_college_count#27)#65] Results [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#56 AS cnt1#66, avg(cd_dep_count#25)#57 AS avg(cd_dep_count)#67, max(cd_dep_count#25)#58 AS max(cd_dep_count)#68, sum(cd_dep_count#25)#59 AS sum(cd_dep_count)#69, cd_dep_employed_count#26, count(1)#56 AS cnt2#70, avg(cd_dep_employed_count#26)#60 AS avg(cd_dep_employed_count)#71, max(cd_dep_employed_count#26)#61 AS max(cd_dep_employed_count)#72, sum(cd_dep_employed_count#26)#62 AS sum(cd_dep_employed_count)#73, cd_dep_college_count#27, count(1)#56 AS cnt3#74, avg(cd_dep_college_count#27)#63 AS avg(cd_dep_college_count)#75, max(cd_dep_college_count#27)#64 AS max(cd_dep_college_count)#76, sum(cd_dep_college_count#27)#65 AS sum(cd_dep_college_count)#77] -(44) TakeOrderedAndProject +(47) TakeOrderedAndProject Input [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (45) +ReusedExchange (48) -(45) ReusedExchange [Reuses operator id: 10] +(48) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#7] -Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index 1ab7d548e59dd..ae82b13a9533e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customsk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -21,11 +21,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -40,19 +41,21 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [cs_ship_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt index e04148fad545e..849d2f418a9a8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (94) -+- * HashAggregate (93) - +- Exchange (92) - +- * HashAggregate (91) - +- Union (90) - :- * HashAggregate (79) - : +- Exchange (78) - : +- * HashAggregate (77) - : +- Union (76) +TakeOrderedAndProject (95) ++- * HashAggregate (94) + +- Exchange (93) + +- * HashAggregate (92) + +- Union (91) + :- * HashAggregate (80) + : +- Exchange (79) + : +- * HashAggregate (78) + : +- Union (77) : :- * HashAggregate (25) : : +- Exchange (24) : : +- * HashAggregate (23) @@ -54,45 +54,46 @@ TakeOrderedAndProject (94) : : : +- * ColumnarToRow (36) : : : +- Scan parquet default.catalog_page (35) : : +- ReusedExchange (41) - : +- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- * Project (72) - : +- * BroadcastHashJoin Inner BuildRight (71) - : :- * Project (69) - : : +- * BroadcastHashJoin Inner BuildRight (68) - : : :- Union (63) + : +- * HashAggregate (76) + : +- Exchange (75) + : +- * HashAggregate (74) + : +- * Project (73) + : +- * BroadcastHashJoin Inner BuildRight (72) + : :- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- Union (64) : : : :- * Project (50) : : : : +- * Filter (49) : : : : +- * ColumnarToRow (48) : : : : +- Scan parquet default.web_sales (47) - : : : +- * Project (62) - : : : +- * SortMergeJoin Inner (61) - : : : :- * Sort (54) - : : : : +- Exchange (53) - : : : : +- * ColumnarToRow (52) - : : : : +- Scan parquet default.web_returns (51) - : : : +- * Sort (60) - : : : +- Exchange (59) - : : : +- * Project (58) - : : : +- * Filter (57) - : : : +- * ColumnarToRow (56) - : : : +- Scan parquet default.web_sales (55) - : : +- BroadcastExchange (67) - : : +- * Filter (66) - : : +- * ColumnarToRow (65) - : : +- Scan parquet default.web_site (64) - : +- ReusedExchange (70) - :- * HashAggregate (84) - : +- Exchange (83) - : +- * HashAggregate (82) - : +- * HashAggregate (81) - : +- ReusedExchange (80) - +- * HashAggregate (89) - +- Exchange (88) - +- * HashAggregate (87) - +- * HashAggregate (86) - +- ReusedExchange (85) + : : : +- * Project (63) + : : : +- * SortMergeJoin Inner (62) + : : : :- * Sort (55) + : : : : +- Exchange (54) + : : : : +- * Filter (53) + : : : : +- * ColumnarToRow (52) + : : : : +- Scan parquet default.web_returns (51) + : : : +- * Sort (61) + : : : +- Exchange (60) + : : : +- * Project (59) + : : : +- * Filter (58) + : : : +- * ColumnarToRow (57) + : : : +- Scan parquet default.web_sales (56) + : : +- BroadcastExchange (68) + : : +- * Filter (67) + : : +- * ColumnarToRow (66) + : : +- Scan parquet default.web_site (65) + : +- ReusedExchange (71) + :- * HashAggregate (85) + : +- Exchange (84) + : +- * HashAggregate (83) + : +- * HashAggregate (82) + : +- ReusedExchange (81) + +- * HashAggregate (90) + +- Exchange (89) + +- * HashAggregate (88) + +- * HashAggregate (87) + +- ReusedExchange (86) (1) Scan parquet default.store_sales @@ -173,7 +174,7 @@ Input [2]: [d_date_sk#25, d_date#26] (18) Filter [codegen id : 4] Input [2]: [d_date_sk#25, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 10442)) AND (d_date#26 <= 10456)) AND isnotnull(d_date_sk#25)) +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 1998-08-04)) AND (d_date#26 <= 1998-08-18)) AND isnotnull(d_date_sk#25)) (19) Project [codegen id : 4] Output [1]: [d_date_sk#25] @@ -331,229 +332,234 @@ Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#1 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -(53) Exchange +(53) Filter [codegen id : 14] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Condition : (isnotnull(wr_item_sk#98) AND isnotnull(wr_order_number#99)) + +(54) Exchange Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] -(54) Sort [codegen id : 15] +(55) Sort [codegen id : 15] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 -(55) Scan parquet default.web_sales +(56) Scan parquet default.web_sales Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 16] +(57) ColumnarToRow [codegen id : 16] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(57) Filter [codegen id : 16] +(58) Filter [codegen id : 16] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) -(58) Project [codegen id : 16] +(59) Project [codegen id : 16] Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(59) Exchange +(60) Exchange Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] -(60) Sort [codegen id : 17] +(61) Sort [codegen id : 17] Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 -(61) SortMergeJoin [codegen id : 18] +(62) SortMergeJoin [codegen id : 18] Left keys [2]: [wr_item_sk#98, wr_order_number#99] Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None -(62) Project [codegen id : 18] +(63) Project [codegen id : 18] Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -(63) Union +(64) Union -(64) Scan parquet default.web_site +(65) Scan parquet default.web_site Output [2]: [web_site_sk#113, web_site_id#114] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 19] +(66) ColumnarToRow [codegen id : 19] Input [2]: [web_site_sk#113, web_site_id#114] -(66) Filter [codegen id : 19] +(67) Filter [codegen id : 19] Input [2]: [web_site_sk#113, web_site_id#114] Condition : isnotnull(web_site_sk#113) -(67) BroadcastExchange +(68) BroadcastExchange Input [2]: [web_site_sk#113, web_site_id#114] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] -(68) BroadcastHashJoin [codegen id : 21] +(69) BroadcastHashJoin [codegen id : 21] Left keys [1]: [wsr_web_site_sk#92] Right keys [1]: [web_site_sk#113] Join condition: None -(69) Project [codegen id : 21] +(70) Project [codegen id : 21] Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] -(70) ReusedExchange [Reuses operator id: 20] +(71) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] -(71) BroadcastHashJoin [codegen id : 21] +(72) BroadcastHashJoin [codegen id : 21] Left keys [1]: [date_sk#93] Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None -(72) Project [codegen id : 21] +(73) Project [codegen id : 21] Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] -(73) HashAggregate [codegen id : 21] +(74) HashAggregate [codegen id : 21] Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Keys [1]: [web_site_id#114] Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -(74) Exchange +(75) Exchange Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] -(75) HashAggregate [codegen id : 22] +(76) HashAggregate [codegen id : 22] Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] Keys [1]: [web_site_id#114] Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] Results [5]: [web channel AS channel#129, concat(web_site, web_site_id#114) AS id#130, MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#131, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS returns#132, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#133] -(76) Union +(77) Union -(77) HashAggregate [codegen id : 23] +(78) HashAggregate [codegen id : 23] Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] Aggregate Attributes [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] Results [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -(78) Exchange +(79) Exchange Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#146] -(79) HashAggregate [codegen id : 24] +(80) HashAggregate [codegen id : 24] Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] Aggregate Attributes [3]: [sum(sales#43)#147, sum(returns#44)#148, sum(profit#45)#149] Results [5]: [channel#41, id#42, cast(sum(sales#43)#147 as decimal(37,2)) AS sales#150, cast(sum(returns#44)#148 as decimal(37,2)) AS returns#151, cast(sum(profit#45)#149 as decimal(38,2)) AS profit#152] -(80) ReusedExchange [Reuses operator id: unknown] +(81) ReusedExchange [Reuses operator id: unknown] Output [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] -(81) HashAggregate [codegen id : 48] +(82) HashAggregate [codegen id : 48] Input [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#159)] Aggregate Attributes [3]: [sum(sales#43)#160, sum(returns#44)#161, sum(profit#159)#162] Results [4]: [channel#41, sum(sales#43)#160 AS sales#163, sum(returns#44)#161 AS returns#164, sum(profit#159)#162 AS profit#165] -(82) HashAggregate [codegen id : 48] +(83) HashAggregate [codegen id : 48] Input [4]: [channel#41, sales#163, returns#164, profit#165] Keys [1]: [channel#41] Functions [3]: [partial_sum(sales#163), partial_sum(returns#164), partial_sum(profit#165)] Aggregate Attributes [6]: [sum#166, isEmpty#167, sum#168, isEmpty#169, sum#170, isEmpty#171] Results [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] -(83) Exchange +(84) Exchange Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#178] -(84) HashAggregate [codegen id : 49] +(85) HashAggregate [codegen id : 49] Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] Keys [1]: [channel#41] Functions [3]: [sum(sales#163), sum(returns#164), sum(profit#165)] Aggregate Attributes [3]: [sum(sales#163)#179, sum(returns#164)#180, sum(profit#165)#181] Results [5]: [channel#41, null AS id#182, sum(sales#163)#179 AS sum(sales)#183, sum(returns#164)#180 AS sum(returns)#184, sum(profit#165)#181 AS sum(profit)#185] -(85) ReusedExchange [Reuses operator id: unknown] +(86) ReusedExchange [Reuses operator id: unknown] Output [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] -(86) HashAggregate [codegen id : 73] +(87) HashAggregate [codegen id : 73] Input [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#192)] Aggregate Attributes [3]: [sum(sales#43)#193, sum(returns#44)#194, sum(profit#192)#195] Results [3]: [sum(sales#43)#193 AS sales#163, sum(returns#44)#194 AS returns#164, sum(profit#192)#195 AS profit#165] -(87) HashAggregate [codegen id : 73] +(88) HashAggregate [codegen id : 73] Input [3]: [sales#163, returns#164, profit#165] Keys: [] Functions [3]: [partial_sum(sales#163), partial_sum(returns#164), partial_sum(profit#165)] Aggregate Attributes [6]: [sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201] Results [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] -(88) Exchange +(89) Exchange Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#208] -(89) HashAggregate [codegen id : 74] +(90) HashAggregate [codegen id : 74] Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] Keys: [] Functions [3]: [sum(sales#163), sum(returns#164), sum(profit#165)] Aggregate Attributes [3]: [sum(sales#163)#209, sum(returns#164)#210, sum(profit#165)#211] Results [5]: [null AS channel#212, null AS id#213, sum(sales#163)#209 AS sum(sales)#214, sum(returns#164)#210 AS sum(returns)#215, sum(profit#165)#211 AS sum(profit)#216] -(90) Union +(91) Union -(91) HashAggregate [codegen id : 75] +(92) HashAggregate [codegen id : 75] Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Functions: [] Aggregate Attributes: [] Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -(92) Exchange +(93) Exchange Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Arguments: hashpartitioning(channel#41, id#42, sales#150, returns#151, profit#152, 5), ENSURE_REQUIREMENTS, [id=#217] -(93) HashAggregate [codegen id : 76] +(94) HashAggregate [codegen id : 76] Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Functions: [] Aggregate Attributes: [] Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -(94) TakeOrderedAndProject +(95) TakeOrderedAndProject Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#150, returns#151, profit#152] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5 -ReusedExchange (95) +ReusedExchange (96) -(95) ReusedExchange [Reuses operator id: 20] +(96) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#49 IN dynamicpruning#50 -ReusedExchange (96) +ReusedExchange (97) -(96) ReusedExchange [Reuses operator id: 20] +(97) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt index dd2c25b96568b..9a3d775ea8266 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt @@ -121,10 +121,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Exchange [wr_item_sk,wr_order_number] #9 WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [wr_item_sk,wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter WholeStageCodegen (17) Sort [ws_item_sk,ws_order_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 36b1ff63b2065..a5e7d0339212f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (91) -+- * HashAggregate (90) - +- Exchange (89) - +- * HashAggregate (88) - +- Union (87) - :- * HashAggregate (76) - : +- Exchange (75) - : +- * HashAggregate (74) - : +- Union (73) +TakeOrderedAndProject (92) ++- * HashAggregate (91) + +- Exchange (90) + +- * HashAggregate (89) + +- Union (88) + :- * HashAggregate (77) + : +- Exchange (76) + : +- * HashAggregate (75) + : +- Union (74) : :- * HashAggregate (25) : : +- Exchange (24) : : +- * HashAggregate (23) @@ -54,42 +54,43 @@ TakeOrderedAndProject (91) : : +- * Filter (40) : : +- * ColumnarToRow (39) : : +- Scan parquet default.catalog_page (38) - : +- * HashAggregate (72) - : +- Exchange (71) - : +- * HashAggregate (70) - : +- * Project (69) - : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- Union (60) + : +- * HashAggregate (73) + : +- Exchange (72) + : +- * HashAggregate (71) + : +- * Project (70) + : +- * BroadcastHashJoin Inner BuildRight (69) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- Union (61) : : : :- * Project (50) : : : : +- * Filter (49) : : : : +- * ColumnarToRow (48) : : : : +- Scan parquet default.web_sales (47) - : : : +- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildLeft (58) - : : : :- BroadcastExchange (53) - : : : : +- * ColumnarToRow (52) - : : : : +- Scan parquet default.web_returns (51) - : : : +- * Project (57) - : : : +- * Filter (56) - : : : +- * ColumnarToRow (55) - : : : +- Scan parquet default.web_sales (54) - : : +- ReusedExchange (61) - : +- BroadcastExchange (67) - : +- * Filter (66) - : +- * ColumnarToRow (65) - : +- Scan parquet default.web_site (64) - :- * HashAggregate (81) - : +- Exchange (80) - : +- * HashAggregate (79) - : +- * HashAggregate (78) - : +- ReusedExchange (77) - +- * HashAggregate (86) - +- Exchange (85) - +- * HashAggregate (84) - +- * HashAggregate (83) - +- ReusedExchange (82) + : : : +- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildLeft (59) + : : : :- BroadcastExchange (54) + : : : : +- * Filter (53) + : : : : +- * ColumnarToRow (52) + : : : : +- Scan parquet default.web_returns (51) + : : : +- * Project (58) + : : : +- * Filter (57) + : : : +- * ColumnarToRow (56) + : : : +- Scan parquet default.web_sales (55) + : : +- ReusedExchange (62) + : +- BroadcastExchange (68) + : +- * Filter (67) + : +- * ColumnarToRow (66) + : +- Scan parquet default.web_site (65) + :- * HashAggregate (82) + : +- Exchange (81) + : +- * HashAggregate (80) + : +- * HashAggregate (79) + : +- ReusedExchange (78) + +- * HashAggregate (87) + +- Exchange (86) + +- * HashAggregate (85) + +- * HashAggregate (84) + +- ReusedExchange (83) (1) Scan parquet default.store_sales @@ -143,7 +144,7 @@ Input [2]: [d_date_sk#22, d_date#23] (12) Filter [codegen id : 3] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 10442)) AND (d_date#23 <= 10456)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) (13) Project [codegen id : 3] Output [1]: [d_date_sk#22] @@ -328,217 +329,222 @@ Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#1 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -(53) BroadcastExchange +(53) Filter [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] +Condition : (isnotnull(wr_item_sk#98) AND isnotnull(wr_order_number#99)) -(54) Scan parquet default.web_sales +(54) BroadcastExchange +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false], input[1, bigint, false]),false), [id=#103] + +(55) Scan parquet default.web_sales Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(55) ColumnarToRow +(56) ColumnarToRow Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(56) Filter +(57) Filter Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) -(57) Project +(58) Project Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(58) BroadcastHashJoin [codegen id : 15] +(59) BroadcastHashJoin [codegen id : 15] Left keys [2]: [wr_item_sk#98, wr_order_number#99] Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None -(59) Project [codegen id : 15] +(60) Project [codegen id : 15] Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -(60) Union +(61) Union -(61) ReusedExchange [Reuses operator id: 14] +(62) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] -(62) BroadcastHashJoin [codegen id : 18] +(63) BroadcastHashJoin [codegen id : 18] Left keys [1]: [date_sk#93] Right keys [1]: [cast(d_date_sk#22 as bigint)] Join condition: None -(63) Project [codegen id : 18] +(64) Project [codegen id : 18] Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] -(64) Scan parquet default.web_site +(65) Scan parquet default.web_site Output [2]: [web_site_sk#112, web_site_id#113] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 17] +(66) ColumnarToRow [codegen id : 17] Input [2]: [web_site_sk#112, web_site_id#113] -(66) Filter [codegen id : 17] +(67) Filter [codegen id : 17] Input [2]: [web_site_sk#112, web_site_id#113] Condition : isnotnull(web_site_sk#112) -(67) BroadcastExchange +(68) BroadcastExchange Input [2]: [web_site_sk#112, web_site_id#113] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] -(68) BroadcastHashJoin [codegen id : 18] +(69) BroadcastHashJoin [codegen id : 18] Left keys [1]: [wsr_web_site_sk#92] Right keys [1]: [web_site_sk#112] Join condition: None -(69) Project [codegen id : 18] +(70) Project [codegen id : 18] Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] -(70) HashAggregate [codegen id : 18] +(71) HashAggregate [codegen id : 18] Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] Keys [1]: [web_site_id#113] Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -(71) Exchange +(72) Exchange Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] -(72) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] Keys [1]: [web_site_id#113] Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] Results [5]: [web channel AS channel#128, concat(web_site, web_site_id#113) AS id#129, MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#130, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS returns#131, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#132] -(73) Union +(74) Union -(74) HashAggregate [codegen id : 20] +(75) HashAggregate [codegen id : 20] Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] Results [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -(75) Exchange +(76) Exchange Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#145] -(76) HashAggregate [codegen id : 21] +(77) HashAggregate [codegen id : 21] Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] Aggregate Attributes [3]: [sum(sales#43)#146, sum(returns#44)#147, sum(profit#45)#148] Results [5]: [channel#41, id#42, cast(sum(sales#43)#146 as decimal(37,2)) AS sales#149, cast(sum(returns#44)#147 as decimal(37,2)) AS returns#150, cast(sum(profit#45)#148 as decimal(38,2)) AS profit#151] -(77) ReusedExchange [Reuses operator id: unknown] +(78) ReusedExchange [Reuses operator id: unknown] Output [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] -(78) HashAggregate [codegen id : 42] +(79) HashAggregate [codegen id : 42] Input [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#158)] Aggregate Attributes [3]: [sum(sales#43)#159, sum(returns#44)#160, sum(profit#158)#161] Results [4]: [channel#41, sum(sales#43)#159 AS sales#162, sum(returns#44)#160 AS returns#163, sum(profit#158)#161 AS profit#164] -(79) HashAggregate [codegen id : 42] +(80) HashAggregate [codegen id : 42] Input [4]: [channel#41, sales#162, returns#163, profit#164] Keys [1]: [channel#41] Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] Aggregate Attributes [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] Results [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -(80) Exchange +(81) Exchange Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#177] -(81) HashAggregate [codegen id : 43] +(82) HashAggregate [codegen id : 43] Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Keys [1]: [channel#41] Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] Aggregate Attributes [3]: [sum(sales#162)#178, sum(returns#163)#179, sum(profit#164)#180] Results [5]: [channel#41, null AS id#181, sum(sales#162)#178 AS sum(sales)#182, sum(returns#163)#179 AS sum(returns)#183, sum(profit#164)#180 AS sum(profit)#184] -(82) ReusedExchange [Reuses operator id: unknown] +(83) ReusedExchange [Reuses operator id: unknown] Output [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] -(83) HashAggregate [codegen id : 64] +(84) HashAggregate [codegen id : 64] Input [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#191)] Aggregate Attributes [3]: [sum(sales#43)#192, sum(returns#44)#193, sum(profit#191)#194] Results [3]: [sum(sales#43)#192 AS sales#162, sum(returns#44)#193 AS returns#163, sum(profit#191)#194 AS profit#164] -(84) HashAggregate [codegen id : 64] +(85) HashAggregate [codegen id : 64] Input [3]: [sales#162, returns#163, profit#164] Keys: [] Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] Aggregate Attributes [6]: [sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200] Results [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] -(85) Exchange +(86) Exchange Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#207] -(86) HashAggregate [codegen id : 65] +(87) HashAggregate [codegen id : 65] Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Keys: [] Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] Aggregate Attributes [3]: [sum(sales#162)#208, sum(returns#163)#209, sum(profit#164)#210] Results [5]: [null AS channel#211, null AS id#212, sum(sales#162)#208 AS sum(sales)#213, sum(returns#163)#209 AS sum(returns)#214, sum(profit#164)#210 AS sum(profit)#215] -(87) Union +(88) Union -(88) HashAggregate [codegen id : 66] +(89) HashAggregate [codegen id : 66] Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Functions: [] Aggregate Attributes: [] Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -(89) Exchange +(90) Exchange Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Arguments: hashpartitioning(channel#41, id#42, sales#149, returns#150, profit#151, 5), ENSURE_REQUIREMENTS, [id=#216] -(90) HashAggregate [codegen id : 67] +(91) HashAggregate [codegen id : 67] Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Functions: [] Aggregate Attributes: [] Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -(91) TakeOrderedAndProject +(92) TakeOrderedAndProject Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#149, returns#150, profit#151] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5 -ReusedExchange (92) +ReusedExchange (93) -(92) ReusedExchange [Reuses operator id: 14] +(93) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#49 IN dynamicpruning#50 -ReusedExchange (93) +ReusedExchange (94) -(93) ReusedExchange [Reuses operator id: 14] +(94) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index caa7fcf4957ca..a787e89a028aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -118,10 +118,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #9 WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [wr_item_sk,wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 Project [ws_item_sk,ws_web_site_sk,ws_order_number] Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt index 3c65529504320..09545881f17c3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt @@ -111,7 +111,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#8, s_county#9, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -119,7 +119,7 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Condition : isnotnull(s_store_sk#8) +Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_state#10)) (14) Scan parquet default.store_sales Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Condition : ((isnotnull(ranking#19) AND (ranking#19 <= 5)) AND isnotnull(s_state#16)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt index c1b7cfef5638c..00cad8c468868 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk] + Filter [s_store_sk,s_state] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_county,s_state] @@ -51,7 +51,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #6 WholeStageCodegen (7) Project [s_state] - Filter [ranking] + Filter [ranking,s_state] InputAdapter Window [_w2,s_state] WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index 406acb0e0a27f..de78b4ff4cdcc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -111,7 +111,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#8, s_county#9, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -119,7 +119,7 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Condition : isnotnull(s_store_sk#8) +Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_state#10)) (14) Scan parquet default.store_sales Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Condition : ((isnotnull(ranking#19) AND (ranking#19 <= 5)) AND isnotnull(s_state#16)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index cd1e7e69e65d0..b6fdfb5868b9c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk] + Filter [s_store_sk,s_state] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_county,s_state] @@ -51,7 +51,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #6 WholeStageCodegen (7) Project [s_state] - Filter [ranking] + Filter [ranking,s_state] InputAdapter Window [_w2,s_state] WholeStageCodegen (6) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index bb141ab885076..5f3df05bdf07c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -1430,7 +1430,7 @@ abstract class DynamicPartitionPruningSuiteBase .mode("overwrite") .saveAsTable("df5") - Given(s"Inferred DPP on partition column") + Given("Inferred DPP on partition column") Seq(true, false).foreach { infer => withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> s"$infer") { val df = sql( @@ -1450,48 +1450,26 @@ abstract class DynamicPartitionPruningSuiteBase } else { assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 1) } - checkDistinctSubqueries(df, 1) - checkPartitionPruningPredicate(df, false, true) - assert(!checkUnpushedFilters(df)) - checkAnswer(df, Row(0, 0) :: Row(1, 1) :: Nil) } } - Given("Remove no benefit inferred DPP on partition column " + - s"when ${SQLConf.EXCHANGE_REUSE_ENABLED.key} is disabled") - Seq(true, false).foreach { reuse => - Seq(true, false).foreach { broadcastOnly => - withSQLConf( - SQLConf.EXCHANGE_REUSE_ENABLED.key -> s"$reuse", - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> s"$broadcastOnly") { - val df = sql( - """ - |SELECT t1.id, - | df4.k - |FROM (SELECT df3.id, - | df1.k - | FROM df1 - | JOIN df3 - | ON df1.k = df3.k) t1 - | JOIN df4 - | ON t1.k = df4.k AND df4.id < 2 - |""".stripMargin) - - if (!reuse) { - assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 1) - checkDistinctSubqueries(df, 0) - checkPartitionPruningPredicate(df, !broadcastOnly, false) - } else { - assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 2) - checkDistinctSubqueries(df, 1) - checkPartitionPruningPredicate(df, false, true) - } - assert(!checkUnpushedFilters(df)) - - checkAnswer(df, Row(0, 0) :: Row(1, 1) :: Nil) - } - } + Given("Remove no benefit inferred DPP on partition column") + withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "true") { + val df = sql( + """ + |SELECT t1.id, + | df4.k + |FROM (SELECT df3.id, + | df1.k + | FROM df1 + | JOIN df3 + | ON df1.k = df3.k) t1 + | JOIN df4 + | ON t1.k = df4.k AND df4.id < 2 + |""".stripMargin) + assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 1) + checkAnswer(df, Row(0, 0) :: Row(1, 1) :: Nil) } Given("Remove inferred DPP on non-partition column") @@ -1510,10 +1488,6 @@ abstract class DynamicPartitionPruningSuiteBase |""".stripMargin) assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 1) - checkDistinctSubqueries(df, 1) - checkPartitionPruningPredicate(df, false, true) - assert(!checkUnpushedFilters(df)) - checkAnswer(df, Row(0, 0) :: Row(1, 1) :: Nil) } } From ba71ead328c477c7c1615f5623095f73d6c16758 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sat, 13 Feb 2021 16:16:53 +0800 Subject: [PATCH 8/9] fix --- .../sql/catalyst/optimizer/Optimizer.scala | 55 +- .../InferDynamicPruningFilters.scala | 78 ++- .../approved-plans-v1_4/q10.sf100/explain.txt | 214 ++++--- .../q10.sf100/simplified.txt | 13 +- .../approved-plans-v1_4/q10/explain.txt | 190 +++---- .../approved-plans-v1_4/q10/simplified.txt | 13 +- .../q14a.sf100/explain.txt | 4 +- .../q14a.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q14a/explain.txt | 4 +- .../approved-plans-v1_4/q14a/simplified.txt | 2 +- .../q14b.sf100/explain.txt | 4 +- .../q14b.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q14b/explain.txt | 4 +- .../approved-plans-v1_4/q14b/simplified.txt | 2 +- .../approved-plans-v1_4/q16.sf100/explain.txt | 172 +++--- .../q16.sf100/simplified.txt | 16 +- .../approved-plans-v1_4/q16/explain.txt | 172 +++--- .../approved-plans-v1_4/q16/simplified.txt | 16 +- .../q23a.sf100/explain.txt | 522 +++++++++--------- .../q23a.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q23a/explain.txt | 418 +++++++------- .../approved-plans-v1_4/q23a/simplified.txt | 20 +- .../q23b.sf100/explain.txt | 8 +- .../q23b.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q23b/explain.txt | 8 +- .../approved-plans-v1_4/q23b/simplified.txt | 4 +- .../approved-plans-v1_4/q33.sf100/explain.txt | 8 +- .../q33.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q33/explain.txt | 8 +- .../approved-plans-v1_4/q33/simplified.txt | 4 +- .../approved-plans-v1_4/q35.sf100/explain.txt | 222 ++++---- .../q35.sf100/simplified.txt | 13 +- .../approved-plans-v1_4/q35/explain.txt | 186 +++---- .../approved-plans-v1_4/q35/simplified.txt | 13 +- .../approved-plans-v1_4/q5.sf100/explain.txt | 134 +++-- .../q5.sf100/simplified.txt | 9 +- .../approved-plans-v1_4/q5/explain.txt | 131 +++-- .../approved-plans-v1_4/q5/simplified.txt | 9 +- .../approved-plans-v1_4/q56.sf100/explain.txt | 8 +- .../q56.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q56/explain.txt | 8 +- .../approved-plans-v1_4/q56/simplified.txt | 4 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 367 ++++++------ .../q58.sf100/simplified.txt | 36 +- .../approved-plans-v1_4/q58/explain.txt | 369 ++++++------- .../approved-plans-v1_4/q58/simplified.txt | 44 +- .../approved-plans-v1_4/q60.sf100/explain.txt | 8 +- .../q60.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q60/explain.txt | 8 +- .../approved-plans-v1_4/q60/simplified.txt | 4 +- .../approved-plans-v1_4/q69.sf100/explain.txt | 210 ++++--- .../q69.sf100/simplified.txt | 31 +- .../approved-plans-v1_4/q69/explain.txt | 198 +++---- .../approved-plans-v1_4/q69/simplified.txt | 31 +- .../approved-plans-v1_4/q70.sf100/explain.txt | 6 +- .../q70.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q70/explain.txt | 6 +- .../approved-plans-v1_4/q70/simplified.txt | 4 +- .../approved-plans-v1_4/q83.sf100/explain.txt | 496 ++++++++--------- .../q83.sf100/simplified.txt | 16 +- .../approved-plans-v1_4/q83/explain.txt | 474 ++++++++-------- .../approved-plans-v1_4/q83/simplified.txt | 28 +- .../approved-plans-v1_4/q93.sf100/explain.txt | 46 +- .../q93.sf100/simplified.txt | 7 +- .../approved-plans-v1_4/q93/explain.txt | 98 ++-- .../approved-plans-v1_4/q93/simplified.txt | 7 +- .../approved-plans-v1_4/q94.sf100/explain.txt | 184 +++--- .../q94.sf100/simplified.txt | 16 +- .../approved-plans-v1_4/q94/explain.txt | 184 +++--- .../approved-plans-v1_4/q94/simplified.txt | 16 +- .../approved-plans-v1_4/q95.sf100/explain.txt | 4 +- .../q95.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q95/explain.txt | 4 +- .../approved-plans-v1_4/q95/simplified.txt | 2 +- .../q10a.sf100/explain.txt | 202 +++---- .../q10a.sf100/simplified.txt | 31 +- .../approved-plans-v2_7/q10a/explain.txt | 194 +++---- .../approved-plans-v2_7/q10a/simplified.txt | 31 +- .../approved-plans-v2_7/q14.sf100/explain.txt | 4 +- .../q14.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q14/explain.txt | 4 +- .../approved-plans-v2_7/q14/simplified.txt | 2 +- .../q14a.sf100/explain.txt | 4 +- .../q14a.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q14a/explain.txt | 4 +- .../approved-plans-v2_7/q14a/simplified.txt | 2 +- .../approved-plans-v2_7/q35.sf100/explain.txt | 222 ++++---- .../q35.sf100/simplified.txt | 13 +- .../approved-plans-v2_7/q35/explain.txt | 186 +++---- .../approved-plans-v2_7/q35/simplified.txt | 13 +- .../q35a.sf100/explain.txt | 222 ++++---- .../q35a.sf100/simplified.txt | 31 +- .../approved-plans-v2_7/q35a/explain.txt | 190 +++---- .../approved-plans-v2_7/q35a/simplified.txt | 31 +- .../approved-plans-v2_7/q5a.sf100/explain.txt | 186 +++---- .../q5a.sf100/simplified.txt | 9 +- .../approved-plans-v2_7/q5a/explain.txt | 176 +++--- .../approved-plans-v2_7/q5a/simplified.txt | 9 +- .../q70a.sf100/explain.txt | 6 +- .../q70a.sf100/simplified.txt | 4 +- .../approved-plans-v2_7/q70a/explain.txt | 6 +- .../approved-plans-v2_7/q70a/simplified.txt | 4 +- 102 files changed, 3535 insertions(+), 3901 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 70112ce3e60bb..1941c9efa9975 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -921,8 +921,18 @@ object InferFiltersFromGenerate extends Rule[LogicalPlan] { } } -abstract class InferFiltersRule extends Rule[LogicalPlan] - with PredicateHelper with ConstraintHelper { +/** + * Generate a list of additional filters from an operator's existing constraint but remove those + * that are either already part of the operator's condition or are part of the operator's child + * constraints. These filters are currently inserted to the existing conditions in the Filter + * operators and on either side of Join operators. + * + * Note: While this optimization is applicable to a lot of types of join, it primarily benefits + * Inner and LeftSemi joins. + */ +object InferFiltersFromConstraints extends Rule[LogicalPlan] + with PredicateHelper with ConstraintHelper { + def apply(plan: LogicalPlan): LogicalPlan = { if (SQLConf.get.constraintPropagationEnabled) { inferFilters(plan) @@ -931,14 +941,6 @@ abstract class InferFiltersRule extends Rule[LogicalPlan] } } - protected def getBaseConstraints( - left: LogicalPlan, - right: LogicalPlan, - conditionOpt: Option[Expression]): ExpressionSet = { - left.constraints.union(right.constraints) - .union(ExpressionSet(conditionOpt.map(splitConjunctivePredicates).getOrElse(Nil))) - } - private def inferFilters(plan: LogicalPlan): LogicalPlan = plan transform { case filter @ Filter(condition, child) => val newFilters = filter.constraints -- @@ -949,25 +951,25 @@ abstract class InferFiltersRule extends Rule[LogicalPlan] filter } - case join @ Join(left, right, joinType, _, _) => + case join @ Join(left, right, joinType, conditionOpt, _) => joinType match { // For inner join, we can infer additional filters for both sides. LeftSemi is kind of an // inner join, it just drops the right side in the final output. case _: InnerLike | LeftSemi => - val allConstraints = getAllConstraints(join) + val allConstraints = getAllConstraints(left, right, conditionOpt) val newLeft = inferNewFilter(left, allConstraints) val newRight = inferNewFilter(right, allConstraints) join.copy(left = newLeft, right = newRight) // For right outer join, we can only infer additional filters for left side. case RightOuter => - val allConstraints = getAllConstraints(join) + val allConstraints = getAllConstraints(left, right, conditionOpt) val newLeft = inferNewFilter(left, allConstraints) join.copy(left = newLeft) // For left join, we can only infer additional filters for right side. case LeftOuter | LeftAnti => - val allConstraints = getAllConstraints(join) + val allConstraints = getAllConstraints(left, right, conditionOpt) val newRight = inferNewFilter(right, allConstraints) join.copy(right = newRight) @@ -975,7 +977,14 @@ abstract class InferFiltersRule extends Rule[LogicalPlan] } } - protected def getAllConstraints(join: Join): ExpressionSet + private def getAllConstraints( + left: LogicalPlan, + right: LogicalPlan, + conditionOpt: Option[Expression]): ExpressionSet = { + val baseConstraints = left.constraints.union(right.constraints) + .union(ExpressionSet(conditionOpt.map(splitConjunctivePredicates).getOrElse(Nil))) + baseConstraints.union(inferAdditionalConstraints(baseConstraints)) + } private def inferNewFilter(plan: LogicalPlan, constraints: ExpressionSet): LogicalPlan = { val newPredicates = constraints @@ -991,22 +1000,6 @@ abstract class InferFiltersRule extends Rule[LogicalPlan] } } -/** - * Generate a list of additional filters from an operator's existing constraint but remove those - * that are either already part of the operator's condition or are part of the operator's child - * constraints. These filters are currently inserted to the existing conditions in the Filter - * operators and on either side of Join operators. - * - * Note: While this optimization is applicable to a lot of types of join, it primarily benefits - * Inner and LeftSemi joins. - */ -object InferFiltersFromConstraints extends InferFiltersRule { - override def getAllConstraints(join: Join): ExpressionSet = { - val baseConstraints = getBaseConstraints(join.left, join.right, join.condition) - baseConstraints.union(inferAdditionalConstraints(baseConstraints)) - } -} - /** * Combines all adjacent [[Union]] operators into a single [[Union]]. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/InferDynamicPruningFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/InferDynamicPruningFilters.scala index f2ab3e22dcdf2..b92ab41f0169b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/InferDynamicPruningFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/InferDynamicPruningFilters.scala @@ -17,25 +17,79 @@ package org.apache.spark.sql.execution.dynamicpruning -import org.apache.spark.sql.catalyst.expressions.{DynamicPruningSubquery, ExpressionSet} -import org.apache.spark.sql.catalyst.optimizer.InferFiltersRule -import org.apache.spark.sql.catalyst.plans.logical.Join - -object InferDynamicPruningFilters extends InferFiltersRule { - override def getAllConstraints(join: Join): ExpressionSet = { - val baseConstraints = getBaseConstraints(join.left, join.right, join.condition) - val filtered = inferAdditionalConstraints(baseConstraints, true).filter { +import org.apache.spark.sql.catalyst.expressions.{And, DynamicPruningSubquery, ExpressionSet, PredicateHelper} +import org.apache.spark.sql.catalyst.plans.{InnerLike, LeftAnti, LeftOuter, LeftSemi, RightOuter} +import org.apache.spark.sql.catalyst.plans.logical.{ConstraintHelper, Filter, Join, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.dynamicpruning.PartitionPruning._ +import org.apache.spark.sql.internal.SQLConf + +/** + * Similar to InferFiltersFromConstraints, this one only infer DynamicPruning filters. + */ +object InferDynamicPruningFilters extends Rule[LogicalPlan] + with PredicateHelper with ConstraintHelper { + + def apply(plan: LogicalPlan): LogicalPlan = { + if (SQLConf.get.constraintPropagationEnabled) { + inferFilters(plan) + } else { + plan + } + } + + private def inferFilters(plan: LogicalPlan): LogicalPlan = plan transform { + case join @ Join(left, right, joinType, _, _) => + joinType match { + // For inner join, we can infer additional filters for both sides. LeftSemi is kind of an + // inner join, it just drops the right side in the final output. + case _: InnerLike | LeftSemi => + val allConstraints = inferDynamicPrunings(join) + val newLeft = inferNewFilter(left, allConstraints) + val newRight = inferNewFilter(right, allConstraints) + join.copy(left = newLeft, right = newRight) + + // For right outer join, we can only infer additional filters for left side. + case RightOuter => + val allConstraints = inferDynamicPrunings(join) + val newLeft = inferNewFilter(left, allConstraints) + join.copy(left = newLeft) + + // For left join, we can only infer additional filters for right side. + case LeftOuter | LeftAnti => + val allConstraints = inferDynamicPrunings(join) + val newRight = inferNewFilter(right, allConstraints) + join.copy(right = newRight) + + case _ => join + } + } + + def inferDynamicPrunings(join: Join): ExpressionSet = { + val baseConstraints = join.left.constraints.union(join.right.constraints) + .union(ExpressionSet(join.condition.map(splitConjunctivePredicates).getOrElse(Nil))) + inferAdditionalConstraints(baseConstraints, true).filter { case DynamicPruningSubquery( pruningKey, buildQuery, buildKeys, broadcastKeyIndex, _, _) => - PartitionPruning.getPartitionTableScan(pruningKey, join) match { + getPartitionTableScan(pruningKey, join) match { case Some(partScan) => - val otherExpr = buildKeys(broadcastKeyIndex) - PartitionPruning.pruningHasBenefit(pruningKey, partScan, otherExpr, buildQuery) + pruningHasBenefit(pruningKey, partScan, buildKeys(broadcastKeyIndex), buildQuery) case _ => false } case _ => false } - baseConstraints.union(filtered) + } + + private def inferNewFilter(plan: LogicalPlan, dynamicPrunings: ExpressionSet): LogicalPlan = { + val newPredicates = dynamicPrunings + .filter { c => + c.references.nonEmpty && c.references.subsetOf(plan.outputSet) && c.deterministic + } -- plan.constraints + if (newPredicates.isEmpty) { + plan + } else { + Filter(newPredicates.reduce(And), plan) + } } } diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt index 841c7284d9540..c25973f527384 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt @@ -1,67 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (56) -+- * HashAggregate (55) - +- Exchange (54) - +- * HashAggregate (53) - +- * Project (52) - +- * SortMergeJoin Inner (51) - :- * Sort (45) - : +- Exchange (44) - : +- * Project (43) - : +- * BroadcastHashJoin Inner BuildRight (42) - : :- * Project (36) - : : +- * Filter (35) - : : +- SortMergeJoin ExistenceJoin(exists#1) (34) - : : :- SortMergeJoin ExistenceJoin(exists#2) (26) - : : : :- SortMergeJoin LeftSemi (18) +TakeOrderedAndProject (55) ++- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * Project (51) + +- * SortMergeJoin Inner (50) + :- * Sort (44) + : +- Exchange (43) + : +- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (35) + : : +- * Filter (34) + : : +- SortMergeJoin ExistenceJoin(exists#1) (33) + : : :- SortMergeJoin ExistenceJoin(exists#2) (25) + : : : :- SortMergeJoin LeftSemi (17) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- * Sort (17) - : : : : +- Exchange (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Filter (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- Scan parquet default.store_sales (6) - : : : : +- BroadcastExchange (13) - : : : : +- * Project (12) - : : : : +- * Filter (11) - : : : : +- * ColumnarToRow (10) - : : : : +- Scan parquet default.date_dim (9) - : : : +- * Sort (25) - : : : +- Exchange (24) - : : : +- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * ColumnarToRow (20) - : : : : +- Scan parquet default.web_sales (19) - : : : +- ReusedExchange (21) - : : +- * Sort (33) - : : +- Exchange (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- Scan parquet default.catalog_sales (27) - : : +- ReusedExchange (29) - : +- BroadcastExchange (41) - : +- * Project (40) - : +- * Filter (39) - : +- * ColumnarToRow (38) - : +- Scan parquet default.customer_address (37) - +- * Sort (50) - +- Exchange (49) - +- * Filter (48) - +- * ColumnarToRow (47) - +- Scan parquet default.customer_demographics (46) + : : : : +- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Project (14) + : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : :- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (12) + : : : : +- * Project (11) + : : : : +- * Filter (10) + : : : : +- * ColumnarToRow (9) + : : : : +- Scan parquet default.date_dim (8) + : : : +- * Sort (24) + : : : +- Exchange (23) + : : : +- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- Scan parquet default.web_sales (18) + : : : +- ReusedExchange (20) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * ColumnarToRow (27) + : : : +- Scan parquet default.catalog_sales (26) + : : +- ReusedExchange (28) + : +- BroadcastExchange (40) + : +- * Project (39) + : +- * Filter (38) + : +- * ColumnarToRow (37) + : +- Scan parquet default.customer_address (36) + +- * Sort (49) + +- Exchange (48) + +- * Filter (47) + +- * ColumnarToRow (46) + +- Scan parquet default.customer_demographics (45) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -69,7 +68,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -84,241 +83,236 @@ Output [2]: [ss_customer_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -(8) Filter [codegen id : 4] -Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_customer_sk#7) - -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 2002)) AND (d_moy#12 >= 1)) AND (d_moy#12 <= 4)) AND isnotnull(d_date_sk#10)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -(13) BroadcastExchange +(12) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [1]: [ss_customer_sk#7] Input [3]: [ss_customer_sk#7, ss_sold_date_sk#8, d_date_sk#10] -(16) Exchange +(15) Exchange Input [1]: [ss_customer_sk#7] Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [1]: [ss_customer_sk#7] Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(18) SortMergeJoin +(17) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#7] Join condition: None -(19) Scan parquet default.web_sales +(18) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#16), dynamicpruningexpression(ws_sold_date_sk#16 IN dynamicpruning#9)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] -(21) ReusedExchange [Reuses operator id: 13] +(20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -(22) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(23) Project [codegen id : 7] +(22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] -(24) Exchange +(23) Exchange Input [1]: [ws_bill_customer_sk#15] Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] -(25) Sort [codegen id : 8] +(24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] Arguments: [ws_bill_customer_sk#15 ASC NULLS FIRST], false, 0 -(26) SortMergeJoin +(25) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#15] Join condition: None -(27) Scan parquet default.catalog_sales +(26) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 10] +(27) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] -(29) ReusedExchange [Reuses operator id: 13] +(28) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -(30) BroadcastHashJoin [codegen id : 10] +(29) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#19] Right keys [1]: [d_date_sk#10] Join condition: None -(31) Project [codegen id : 10] +(30) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#18] Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] -(32) Exchange +(31) Exchange Input [1]: [cs_ship_customer_sk#18] Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] -(33) Sort [codegen id : 11] +(32) Sort [codegen id : 11] Input [1]: [cs_ship_customer_sk#18] Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#18] Join condition: None -(35) Filter [codegen id : 13] +(34) Filter [codegen id : 13] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(36) Project [codegen id : 13] +(35) Project [codegen id : 13] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(37) Scan parquet default.customer_address +(36) Scan parquet default.customer_address Output [2]: [ca_address_sk#21, ca_county#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(38) ColumnarToRow [codegen id : 12] +(37) ColumnarToRow [codegen id : 12] Input [2]: [ca_address_sk#21, ca_county#22] -(39) Filter [codegen id : 12] +(38) Filter [codegen id : 12] Input [2]: [ca_address_sk#21, ca_county#22] Condition : (ca_county#22 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#21)) -(40) Project [codegen id : 12] +(39) Project [codegen id : 12] Output [1]: [ca_address_sk#21] Input [2]: [ca_address_sk#21, ca_county#22] -(41) BroadcastExchange +(40) BroadcastExchange Input [1]: [ca_address_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] -(42) BroadcastHashJoin [codegen id : 13] +(41) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#21] Join condition: None -(43) Project [codegen id : 13] +(42) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#21] -(44) Exchange +(43) Exchange Input [1]: [c_current_cdemo_sk#4] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#24] -(45) Sort [codegen id : 14] +(44) Sort [codegen id : 14] Input [1]: [c_current_cdemo_sk#4] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(46) Scan parquet default.customer_demographics +(45) Scan parquet default.customer_demographics Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 15] +(46) ColumnarToRow [codegen id : 15] Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -(48) Filter [codegen id : 15] +(47) Filter [codegen id : 15] Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Condition : isnotnull(cd_demo_sk#25) -(49) Exchange +(48) Exchange Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#34] -(50) Sort [codegen id : 16] +(49) Sort [codegen id : 16] Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 -(51) SortMergeJoin [codegen id : 17] +(50) SortMergeJoin [codegen id : 17] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#25] Join condition: None -(52) Project [codegen id : 17] +(51) Project [codegen id : 17] Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -(53) HashAggregate [codegen id : 17] +(52) HashAggregate [codegen id : 17] Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#35] Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -(54) Exchange +(53) Exchange Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] -(55) HashAggregate [codegen id : 18] +(54) HashAggregate [codegen id : 18] Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#38] Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] -(56) TakeOrderedAndProject +(55) TakeOrderedAndProject Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (57) +ReusedExchange (56) -(57) ReusedExchange [Reuses operator id: 13] +(56) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt index fa8f3c241be38..c10844dccad8f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt @@ -26,7 +26,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -37,12 +37,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #5 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 0e011f1c44c0b..07bcd309e9ce6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -1,59 +1,58 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (31) - : : +- * Filter (30) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (29) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (22) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (30) + : : +- * Filter (29) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (28) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (21) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (14) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (14) - : : : : +- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (21) - : : : +- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (18) - : : +- BroadcastExchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * ColumnarToRow (24) - : : : +- Scan parquet default.catalog_sales (23) - : : +- ReusedExchange (25) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.customer_address (32) - +- BroadcastExchange (42) - +- * Filter (41) - +- * ColumnarToRow (40) - +- Scan parquet default.customer_demographics (39) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * ColumnarToRow (5) + : : : : : +- Scan parquet default.store_sales (4) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.date_dim (6) + : : : +- BroadcastExchange (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- BroadcastExchange (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (24) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.customer_address (31) + +- BroadcastExchange (41) + +- * Filter (40) + +- * ColumnarToRow (39) + +- Scan parquet default.customer_demographics (38) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -61,224 +60,219 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) Filter [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_customer_sk#6) - -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(11) BroadcastExchange +(10) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] -(12) BroadcastHashJoin [codegen id : 2] +(11) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join condition: None -(13) Project [codegen id : 2] +(12) Project [codegen id : 2] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(14) BroadcastExchange +(13) BroadcastExchange Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(15) BroadcastHashJoin [codegen id : 9] +(14) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join condition: None -(16) Scan parquet default.web_sales +(15) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#15), dynamicpruningexpression(ws_sold_date_sk#15 IN dynamicpruning#8)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] +(16) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] -(18) ReusedExchange [Reuses operator id: 11] +(17) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -(19) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] Right keys [1]: [d_date_sk#9] Join condition: None -(20) Project [codegen id : 4] +(19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] -(21) BroadcastExchange +(20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(22) BroadcastHashJoin [codegen id : 9] +(21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#14] Join condition: None -(23) Scan parquet default.catalog_sales +(22) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 6] +(23) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] -(25) ReusedExchange [Reuses operator id: 11] +(24) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -(26) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#9] Join condition: None -(27) Project [codegen id : 6] +(26) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#17] Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] -(28) BroadcastExchange +(27) BroadcastExchange Input [1]: [cs_ship_customer_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(29) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#17] Join condition: None -(30) Filter [codegen id : 9] +(29) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(31) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(32) Scan parquet default.customer_address +(31) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 7] +(32) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#20, ca_county#21] -(34) Filter [codegen id : 7] +(33) Filter [codegen id : 7] Input [2]: [ca_address_sk#20, ca_county#21] Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) -(35) Project [codegen id : 7] +(34) Project [codegen id : 7] Output [1]: [ca_address_sk#20] Input [2]: [ca_address_sk#20, ca_county#21] -(36) BroadcastExchange +(35) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(37) BroadcastHashJoin [codegen id : 9] +(36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join condition: None -(38) Project [codegen id : 9] +(37) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] -(39) Scan parquet default.customer_demographics +(38) Scan parquet default.customer_demographics Output [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(40) ColumnarToRow [codegen id : 8] +(39) ColumnarToRow [codegen id : 8] Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(41) Filter [codegen id : 8] +(40) Filter [codegen id : 8] Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Condition : isnotnull(cd_demo_sk#23) -(42) BroadcastExchange +(41) BroadcastExchange Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(43) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join condition: None -(44) Project [codegen id : 9] +(43) Project [codegen id : 9] Output [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(45) HashAggregate [codegen id : 9] +(44) HashAggregate [codegen id : 9] Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#33] Results [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -(46) Exchange +(45) Exchange Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#35] -(47) HashAggregate [codegen id : 10] +(46) HashAggregate [codegen id : 10] Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#36] Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#36 AS cnt1#37, cd_purchase_estimate#27, count(1)#36 AS cnt2#38, cd_credit_rating#28, count(1)#36 AS cnt3#39, cd_dep_count#29, count(1)#36 AS cnt4#40, cd_dep_employed_count#30, count(1)#36 AS cnt5#41, cd_dep_college_count#31, count(1)#36 AS cnt6#42] -(48) TakeOrderedAndProject +(47) TakeOrderedAndProject Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -ReusedExchange (49) +ReusedExchange (48) -(49) ReusedExchange [Reuses operator id: 11] +(48) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 78861b51e4f6d..261bb6152b22a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -23,12 +23,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) 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 05b0c9b7dfff5..d4ab3f60494d9 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 @@ -162,7 +162,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (7) ColumnarToRow [codegen id : 20] @@ -170,7 +170,7 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] 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 341784cbfdeae..4ee1ff31e9beb 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 @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (20) 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,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 8208efcb1755a..85a3a63413f92 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -135,7 +135,7 @@ Condition : isnotnull(ss_item_sk#1) 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), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -143,7 +143,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] 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)) AND isnotnull(i_item_sk#6)) +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index a66dd29c4927d..f9b76560e363c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] 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 3109147596cf8..76f5ac48f96cc 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 @@ -144,7 +144,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (7) ColumnarToRow [codegen id : 20] @@ -152,7 +152,7 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] 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 be55d3b2fecfc..1d5907e319658 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 @@ -75,7 +75,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (20) 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,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 7315c864712eb..01bcb83622844 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -120,7 +120,7 @@ Condition : isnotnull(ss_item_sk#1) 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), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -128,7 +128,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] 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)) AND isnotnull(i_item_sk#6)) +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 592d91ddf08fd..3507d76bcf9b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -67,7 +67,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index 154c74df64b64..9c56438870f64 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -1,59 +1,57 @@ == Physical Plan == -* Sort (48) -+- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- SortMergeJoin LeftAnti (21) - : : : :- * Project (14) - : : : : +- SortMergeJoin LeftSemi (13) +* Sort (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- SortMergeJoin LeftSemi (12) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.catalog_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.catalog_sales (7) - : : : +- * Sort (20) - : : : +- Exchange (19) - : : : +- * Project (18) - : : : +- * Filter (17) - : : : +- * ColumnarToRow (16) - : : : +- Scan parquet default.catalog_returns (15) - : : +- BroadcastExchange (26) - : : +- * Project (25) - : : +- * Filter (24) - : : +- * ColumnarToRow (23) - : : +- Scan parquet default.customer_address (22) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- * ColumnarToRow (30) - : +- Scan parquet default.call_center (29) - +- BroadcastExchange (40) - +- * Project (39) - +- * Filter (38) - +- * ColumnarToRow (37) - +- Scan parquet default.date_dim (36) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * Project (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.catalog_sales (7) + : : : +- * Sort (18) + : : : +- Exchange (17) + : : : +- * Project (16) + : : : +- * ColumnarToRow (15) + : : : +- Scan parquet default.catalog_returns (14) + : : +- BroadcastExchange (24) + : : +- * Project (23) + : : +- * Filter (22) + : : +- * ColumnarToRow (21) + : : +- Scan parquet default.customer_address (20) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- * ColumnarToRow (28) + : +- Scan parquet default.call_center (27) + +- BroadcastExchange (38) + +- * Project (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.date_dim (34) (1) Scan parquet default.catalog_sales Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] +PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -61,7 +59,7 @@ Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho (3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_order_number#5)) AND isnotnull(cs_warehouse_sk#4)) +Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) (4) Project [codegen id : 1] Output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -79,194 +77,184 @@ Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -(9) Filter [codegen id : 3] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -Condition : (isnotnull(cs_order_number#5) AND isnotnull(cs_warehouse_sk#4)) - -(10) Project [codegen id : 3] +(9) Project [codegen id : 3] Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -(11) Exchange +(10) Exchange Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] -(12) Sort [codegen id : 4] +(11) Sort [codegen id : 4] Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin +(12) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#5#11] Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) -(14) Project [codegen id : 5] +(13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(15) Scan parquet default.catalog_returns +(14) Scan parquet default.catalog_returns Output [2]: [cr_order_number#13, cr_returned_date_sk#14] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 6] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] - -(17) Filter [codegen id : 6] +(15) ColumnarToRow [codegen id : 6] Input [2]: [cr_order_number#13, cr_returned_date_sk#14] -Condition : isnotnull(cr_order_number#13) -(18) Project [codegen id : 6] +(16) Project [codegen id : 6] Output [1]: [cr_order_number#13] Input [2]: [cr_order_number#13, cr_returned_date_sk#14] -(19) Exchange +(17) Exchange Input [1]: [cr_order_number#13] Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] -(20) Sort [codegen id : 7] +(18) Sort [codegen id : 7] Input [1]: [cr_order_number#13] Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 -(21) SortMergeJoin +(19) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#13] Join condition: None -(22) Scan parquet default.customer_address +(20) Scan parquet default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 8] +(21) ColumnarToRow [codegen id : 8] Input [2]: [ca_address_sk#16, ca_state#17] -(24) Filter [codegen id : 8] +(22) Filter [codegen id : 8] Input [2]: [ca_address_sk#16, ca_state#17] Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_address_sk#16)) -(25) Project [codegen id : 8] +(23) Project [codegen id : 8] Output [1]: [ca_address_sk#16] Input [2]: [ca_address_sk#16, ca_state#17] -(26) BroadcastExchange +(24) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(27) BroadcastHashJoin [codegen id : 11] +(25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join condition: None -(28) Project [codegen id : 11] +(26) Project [codegen id : 11] Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -(29) Scan parquet default.call_center +(27) Scan parquet default.call_center Output [2]: [cc_call_center_sk#19, cc_county#20] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 9] +(28) ColumnarToRow [codegen id : 9] Input [2]: [cc_call_center_sk#19, cc_county#20] -(31) Filter [codegen id : 9] +(29) Filter [codegen id : 9] Input [2]: [cc_call_center_sk#19, cc_county#20] Condition : ((isnotnull(cc_county#20) AND (cc_county#20 = Williamson County)) AND isnotnull(cc_call_center_sk#19)) -(32) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [1]: [cc_call_center_sk#19] Input [2]: [cc_call_center_sk#19, cc_county#20] -(33) BroadcastExchange +(31) BroadcastExchange Input [1]: [cc_call_center_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(34) BroadcastHashJoin [codegen id : 11] +(32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#19] Join condition: None -(35) Project [codegen id : 11] +(33) Project [codegen id : 11] Output [4]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#19] -(36) Scan parquet default.date_dim +(34) Scan parquet default.date_dim Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 10] +(35) ColumnarToRow [codegen id : 10] Input [2]: [d_date_sk#22, d_date#23] -(38) Filter [codegen id : 10] +(36) Filter [codegen id : 10] Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2002-02-01)) AND (d_date#23 <= 2002-04-02)) AND isnotnull(d_date_sk#22)) -(39) Project [codegen id : 10] +(37) Project [codegen id : 10] Output [1]: [d_date_sk#22] Input [2]: [d_date_sk#22, d_date#23] -(40) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(41) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#22] Join condition: None -(42) Project [codegen id : 11] +(40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#22] -(43) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] Results [3]: [cs_order_number#5, sum#27, sum#28] -(44) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#27, sum#28] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] Results [3]: [cs_order_number#5, sum#27, sum#28] -(45) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#27, sum#28] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] Results [3]: [sum#27, sum#28, count#30] -(46) Exchange +(44) Exchange Input [3]: [sum#27, sum#28, count#30] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] -(47) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 12] Input [3]: [sum#27, sum#28, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] -(48) Sort [codegen id : 12] +(46) Sort [codegen id : 12] Input [3]: [order count #32, total shipping cost #33, total net profit #34] Arguments: [order count #32 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt index 5627d2f61e900..393a5baec4400 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (12) Exchange [cs_order_number] #2 WholeStageCodegen (1) Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_warehouse_sk] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] @@ -35,20 +35,18 @@ WholeStageCodegen (12) Exchange [cs_order_number] #3 WholeStageCodegen (3) Project [cs_warehouse_sk,cs_order_number] - Filter [cs_order_number,cs_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] WholeStageCodegen (7) Sort [cr_order_number] InputAdapter Exchange [cr_order_number] #4 WholeStageCodegen (6) Project [cr_order_number] - Filter [cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 392a265dcff07..dde6a9f564859 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -1,59 +1,57 @@ == Physical Plan == -* Sort (48) -+- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- SortMergeJoin LeftAnti (21) - : : : :- * Project (14) - : : : : +- SortMergeJoin LeftSemi (13) +* Sort (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- SortMergeJoin LeftSemi (12) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.catalog_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.catalog_sales (7) - : : : +- * Sort (20) - : : : +- Exchange (19) - : : : +- * Project (18) - : : : +- * Filter (17) - : : : +- * ColumnarToRow (16) - : : : +- Scan parquet default.catalog_returns (15) - : : +- BroadcastExchange (26) - : : +- * Project (25) - : : +- * Filter (24) - : : +- * ColumnarToRow (23) - : : +- Scan parquet default.date_dim (22) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- * ColumnarToRow (30) - : +- Scan parquet default.customer_address (29) - +- BroadcastExchange (40) - +- * Project (39) - +- * Filter (38) - +- * ColumnarToRow (37) - +- Scan parquet default.call_center (36) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * Project (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.catalog_sales (7) + : : : +- * Sort (18) + : : : +- Exchange (17) + : : : +- * Project (16) + : : : +- * ColumnarToRow (15) + : : : +- Scan parquet default.catalog_returns (14) + : : +- BroadcastExchange (24) + : : +- * Project (23) + : : +- * Filter (22) + : : +- * ColumnarToRow (21) + : : +- Scan parquet default.date_dim (20) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- * ColumnarToRow (28) + : +- Scan parquet default.customer_address (27) + +- BroadcastExchange (38) + +- * Project (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.call_center (34) (1) Scan parquet default.catalog_sales Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] +PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -61,7 +59,7 @@ Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho (3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_order_number#5)) AND isnotnull(cs_warehouse_sk#4)) +Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) (4) Project [codegen id : 1] Output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -79,194 +77,184 @@ Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -(9) Filter [codegen id : 3] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -Condition : (isnotnull(cs_order_number#5) AND isnotnull(cs_warehouse_sk#4)) - -(10) Project [codegen id : 3] +(9) Project [codegen id : 3] Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -(11) Exchange +(10) Exchange Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] -(12) Sort [codegen id : 4] +(11) Sort [codegen id : 4] Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin +(12) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#5#11] Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) -(14) Project [codegen id : 5] +(13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(15) Scan parquet default.catalog_returns +(14) Scan parquet default.catalog_returns Output [2]: [cr_order_number#13, cr_returned_date_sk#14] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 6] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] - -(17) Filter [codegen id : 6] +(15) ColumnarToRow [codegen id : 6] Input [2]: [cr_order_number#13, cr_returned_date_sk#14] -Condition : isnotnull(cr_order_number#13) -(18) Project [codegen id : 6] +(16) Project [codegen id : 6] Output [1]: [cr_order_number#13] Input [2]: [cr_order_number#13, cr_returned_date_sk#14] -(19) Exchange +(17) Exchange Input [1]: [cr_order_number#13] Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] -(20) Sort [codegen id : 7] +(18) Sort [codegen id : 7] Input [1]: [cr_order_number#13] Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 -(21) SortMergeJoin +(19) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#13] Join condition: None -(22) Scan parquet default.date_dim +(20) Scan parquet default.date_dim Output [2]: [d_date_sk#16, d_date#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 8] +(21) ColumnarToRow [codegen id : 8] Input [2]: [d_date_sk#16, d_date#17] -(24) Filter [codegen id : 8] +(22) Filter [codegen id : 8] Input [2]: [d_date_sk#16, d_date#17] Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2002-02-01)) AND (d_date#17 <= 2002-04-02)) AND isnotnull(d_date_sk#16)) -(25) Project [codegen id : 8] +(23) Project [codegen id : 8] Output [1]: [d_date_sk#16] Input [2]: [d_date_sk#16, d_date#17] -(26) BroadcastExchange +(24) BroadcastExchange Input [1]: [d_date_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(27) BroadcastHashJoin [codegen id : 11] +(25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#16] Join condition: None -(28) Project [codegen id : 11] +(26) Project [codegen id : 11] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#16] -(29) Scan parquet default.customer_address +(27) Scan parquet default.customer_address Output [2]: [ca_address_sk#19, ca_state#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 9] +(28) ColumnarToRow [codegen id : 9] Input [2]: [ca_address_sk#19, ca_state#20] -(31) Filter [codegen id : 9] +(29) Filter [codegen id : 9] Input [2]: [ca_address_sk#19, ca_state#20] Condition : ((isnotnull(ca_state#20) AND (ca_state#20 = GA)) AND isnotnull(ca_address_sk#19)) -(32) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [1]: [ca_address_sk#19] Input [2]: [ca_address_sk#19, ca_state#20] -(33) BroadcastExchange +(31) BroadcastExchange Input [1]: [ca_address_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(34) BroadcastHashJoin [codegen id : 11] +(32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#19] Join condition: None -(35) Project [codegen id : 11] +(33) Project [codegen id : 11] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#19] -(36) Scan parquet default.call_center +(34) Scan parquet default.call_center Output [2]: [cc_call_center_sk#22, cc_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 10] +(35) ColumnarToRow [codegen id : 10] Input [2]: [cc_call_center_sk#22, cc_county#23] -(38) Filter [codegen id : 10] +(36) Filter [codegen id : 10] Input [2]: [cc_call_center_sk#22, cc_county#23] Condition : ((isnotnull(cc_county#23) AND (cc_county#23 = Williamson County)) AND isnotnull(cc_call_center_sk#22)) -(39) Project [codegen id : 10] +(37) Project [codegen id : 10] Output [1]: [cc_call_center_sk#22] Input [2]: [cc_call_center_sk#22, cc_county#23] -(40) BroadcastExchange +(38) BroadcastExchange Input [1]: [cc_call_center_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(41) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#22] Join condition: None -(42) Project [codegen id : 11] +(40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#22] -(43) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] Results [3]: [cs_order_number#5, sum#27, sum#28] -(44) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#27, sum#28] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] Results [3]: [cs_order_number#5, sum#27, sum#28] -(45) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#27, sum#28] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] Results [3]: [sum#27, sum#28, count#30] -(46) Exchange +(44) Exchange Input [3]: [sum#27, sum#28, count#30] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] -(47) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 12] Input [3]: [sum#27, sum#28, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] -(48) Sort [codegen id : 12] +(46) Sort [codegen id : 12] Input [3]: [order count #32, total shipping cost #33, total net profit #34] Arguments: [order count #32 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index 90655232d690e..5edd1426dd8f1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (12) Exchange [cs_order_number] #2 WholeStageCodegen (1) Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_warehouse_sk] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] @@ -35,20 +35,18 @@ WholeStageCodegen (12) Exchange [cs_order_number] #3 WholeStageCodegen (3) Project [cs_warehouse_sk,cs_order_number] - Filter [cs_order_number,cs_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] WholeStageCodegen (7) Sort [cr_order_number] InputAdapter Exchange [cr_order_number] #4 WholeStageCodegen (6) Project [cr_order_number] - Filter [cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt index 46acb1a9eebdb..aa96edd4c7fba 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt @@ -1,104 +1,102 @@ == Physical Plan == -* HashAggregate (100) -+- Exchange (99) - +- * HashAggregate (98) - +- Union (97) - :- * Project (60) - : +- * BroadcastHashJoin Inner BuildRight (59) - : :- * Project (53) - : : +- SortMergeJoin LeftSemi (52) - : : :- * Sort (33) - : : : +- Exchange (32) - : : : +- * Project (31) - : : : +- SortMergeJoin LeftSemi (30) - : : : :- * Sort (5) - : : : : +- Exchange (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.catalog_sales (1) - : : : +- * Sort (29) - : : : +- * Project (28) - : : : +- * Filter (27) - : : : +- * HashAggregate (26) - : : : +- * HashAggregate (25) - : : : +- * Project (24) - : : : +- * SortMergeJoin Inner (23) - : : : :- * Sort (17) - : : : : +- Exchange (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Filter (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- Scan parquet default.store_sales (6) - : : : : +- BroadcastExchange (13) - : : : : +- * Project (12) - : : : : +- * Filter (11) - : : : : +- * ColumnarToRow (10) - : : : : +- Scan parquet default.date_dim (9) - : : : +- * Sort (22) - : : : +- Exchange (21) - : : : +- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet default.item (18) - : : +- * Sort (51) - : : +- * Project (50) - : : +- * Filter (49) - : : +- * HashAggregate (48) - : : +- * HashAggregate (47) - : : +- * Project (46) - : : +- * SortMergeJoin Inner (45) - : : :- * Sort (39) - : : : +- Exchange (38) - : : : +- * Project (37) - : : : +- * Filter (36) - : : : +- * ColumnarToRow (35) - : : : +- Scan parquet default.store_sales (34) - : : +- * Sort (44) - : : +- Exchange (43) - : : +- * Filter (42) - : : +- * ColumnarToRow (41) - : : +- Scan parquet default.customer (40) - : +- BroadcastExchange (58) - : +- * Project (57) - : +- * Filter (56) - : +- * ColumnarToRow (55) - : +- Scan parquet default.date_dim (54) - +- * Project (96) - +- * BroadcastHashJoin Inner BuildRight (95) - :- * Project (93) - : +- SortMergeJoin LeftSemi (92) - : :- * Sort (80) - : : +- Exchange (79) - : : +- * Project (78) - : : +- SortMergeJoin LeftSemi (77) - : : :- * Sort (65) - : : : +- Exchange (64) - : : : +- * Filter (63) - : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.web_sales (61) - : : +- * Sort (76) - : : +- * Project (75) - : : +- * Filter (74) - : : +- * HashAggregate (73) - : : +- * HashAggregate (72) - : : +- * Project (71) - : : +- * SortMergeJoin Inner (70) - : : :- * Sort (67) - : : : +- ReusedExchange (66) - : : +- * Sort (69) - : : +- ReusedExchange (68) - : +- * Sort (91) - : +- * Project (90) - : +- * Filter (89) - : +- * HashAggregate (88) - : +- * HashAggregate (87) - : +- * Project (86) - : +- * SortMergeJoin Inner (85) - : :- * Sort (82) - : : +- ReusedExchange (81) - : +- * Sort (84) - : +- ReusedExchange (83) - +- ReusedExchange (94) +* HashAggregate (98) ++- Exchange (97) + +- * HashAggregate (96) + +- Union (95) + :- * Project (59) + : +- * BroadcastHashJoin Inner BuildRight (58) + : :- * Project (52) + : : +- SortMergeJoin LeftSemi (51) + : : :- * Sort (32) + : : : +- Exchange (31) + : : : +- * Project (30) + : : : +- SortMergeJoin LeftSemi (29) + : : : :- * Sort (4) + : : : : +- Exchange (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.catalog_sales (1) + : : : +- * Sort (28) + : : : +- * Project (27) + : : : +- * Filter (26) + : : : +- * HashAggregate (25) + : : : +- * HashAggregate (24) + : : : +- * Project (23) + : : : +- * SortMergeJoin Inner (22) + : : : :- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Project (14) + : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : :- * Filter (7) + : : : : : +- * ColumnarToRow (6) + : : : : : +- Scan parquet default.store_sales (5) + : : : : +- BroadcastExchange (12) + : : : : +- * Project (11) + : : : : +- * Filter (10) + : : : : +- * ColumnarToRow (9) + : : : : +- Scan parquet default.date_dim (8) + : : : +- * Sort (21) + : : : +- Exchange (20) + : : : +- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.item (17) + : : +- * Sort (50) + : : +- * Project (49) + : : +- * Filter (48) + : : +- * HashAggregate (47) + : : +- * HashAggregate (46) + : : +- * Project (45) + : : +- * SortMergeJoin Inner (44) + : : :- * Sort (38) + : : : +- Exchange (37) + : : : +- * Project (36) + : : : +- * Filter (35) + : : : +- * ColumnarToRow (34) + : : : +- Scan parquet default.store_sales (33) + : : +- * Sort (43) + : : +- Exchange (42) + : : +- * Filter (41) + : : +- * ColumnarToRow (40) + : : +- Scan parquet default.customer (39) + : +- BroadcastExchange (57) + : +- * Project (56) + : +- * Filter (55) + : +- * ColumnarToRow (54) + : +- Scan parquet default.date_dim (53) + +- * Project (94) + +- * BroadcastHashJoin Inner BuildRight (93) + :- * Project (91) + : +- SortMergeJoin LeftSemi (90) + : :- * Sort (78) + : : +- Exchange (77) + : : +- * Project (76) + : : +- SortMergeJoin LeftSemi (75) + : : :- * Sort (63) + : : : +- Exchange (62) + : : : +- * ColumnarToRow (61) + : : : +- Scan parquet default.web_sales (60) + : : +- * Sort (74) + : : +- * Project (73) + : : +- * Filter (72) + : : +- * HashAggregate (71) + : : +- * HashAggregate (70) + : : +- * Project (69) + : : +- * SortMergeJoin Inner (68) + : : :- * Sort (65) + : : : +- ReusedExchange (64) + : : +- * Sort (67) + : : +- ReusedExchange (66) + : +- * Sort (89) + : +- * Project (88) + : +- * Filter (87) + : +- * HashAggregate (86) + : +- * HashAggregate (85) + : +- * Project (84) + : +- * SortMergeJoin Inner (83) + : :- * Sort (80) + : : +- ReusedExchange (79) + : +- * Sort (82) + : +- ReusedExchange (81) + +- ReusedExchange (92) (1) Scan parquet default.catalog_sales @@ -106,25 +104,20 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(3) Filter [codegen id : 1] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) - -(4) Exchange +(3) Exchange Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: hashpartitioning(cs_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#7] -(5) Sort [codegen id : 2] +(4) Sort [codegen id : 2] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Scan parquet default.store_sales +(5) Scan parquet default.store_sales Output [2]: [ss_item_sk#8, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -132,422 +125,417 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 4] +(6) ColumnarToRow [codegen id : 4] Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] -(8) Filter [codegen id : 4] +(7) Filter [codegen id : 4] Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] Condition : isnotnull(ss_item_sk#8) -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#11, d_date#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#11, d_date#12, d_year#13] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#11, d_date#12, d_year#13] Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [2]: [d_date_sk#11, d_date#12] Input [3]: [d_date_sk#11, d_date#12, d_year#13] -(13) BroadcastExchange +(12) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [2]: [ss_item_sk#8, d_date#12] Input [4]: [ss_item_sk#8, ss_sold_date_sk#9, d_date_sk#11, d_date#12] -(16) Exchange +(15) Exchange Input [2]: [ss_item_sk#8, d_date#12] Arguments: hashpartitioning(ss_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#15] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [2]: [ss_item_sk#8, d_date#12] Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 -(18) Scan parquet default.item +(17) Scan parquet default.item Output [2]: [i_item_sk#16, i_item_desc#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 6] Input [2]: [i_item_sk#16, i_item_desc#17] -(20) Filter [codegen id : 6] +(19) Filter [codegen id : 6] Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(21) Exchange +(20) Exchange Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: hashpartitioning(i_item_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] -(22) Sort [codegen id : 7] +(21) Sort [codegen id : 7] Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 -(23) SortMergeJoin [codegen id : 8] +(22) SortMergeJoin [codegen id : 8] Left keys [1]: [ss_item_sk#8] Right keys [1]: [i_item_sk#16] Join condition: None -(24) Project [codegen id : 8] +(23) Project [codegen id : 8] Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] -(25) HashAggregate [codegen id : 8] +(24) HashAggregate [codegen id : 8] Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#19, i_item_sk#16, d_date#12] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#20] Results [4]: [substr(i_item_desc#17, 1, 30)#19, i_item_sk#16, d_date#12, count#21] -(26) HashAggregate [codegen id : 8] +(25) HashAggregate [codegen id : 8] Input [4]: [substr(i_item_desc#17, 1, 30)#19, i_item_sk#16, d_date#12, count#21] Keys [3]: [substr(i_item_desc#17, 1, 30)#19, i_item_sk#16, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#22] Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#22 AS count(1)#24] -(27) Filter [codegen id : 8] +(26) Filter [codegen id : 8] Input [2]: [item_sk#23, count(1)#24] Condition : (count(1)#24 > 4) -(28) Project [codegen id : 8] +(27) Project [codegen id : 8] Output [1]: [item_sk#23] Input [2]: [item_sk#23, count(1)#24] -(29) Sort [codegen id : 8] +(28) Sort [codegen id : 8] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 -(30) SortMergeJoin +(29) SortMergeJoin Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#23] Join condition: None -(31) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(32) Exchange +(31) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#25] -(33) Sort [codegen id : 10] +(32) Sort [codegen id : 10] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(34) Scan parquet default.store_sales +(33) Scan parquet default.store_sales Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 11] +(34) ColumnarToRow [codegen id : 11] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -(36) Filter [codegen id : 11] +(35) Filter [codegen id : 11] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Condition : isnotnull(ss_customer_sk#26) -(37) Project [codegen id : 11] +(36) Project [codegen id : 11] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -(38) Exchange +(37) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#29] -(39) Sort [codegen id : 12] +(38) Sort [codegen id : 12] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 -(40) Scan parquet default.customer +(39) Scan parquet default.customer Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 13] +(40) ColumnarToRow [codegen id : 13] Input [1]: [c_customer_sk#30] -(42) Filter [codegen id : 13] +(41) Filter [codegen id : 13] Input [1]: [c_customer_sk#30] Condition : isnotnull(c_customer_sk#30) -(43) Exchange +(42) Exchange Input [1]: [c_customer_sk#30] Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#31] -(44) Sort [codegen id : 14] +(43) Sort [codegen id : 14] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(45) SortMergeJoin [codegen id : 15] +(44) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#30] Join condition: None -(46) Project [codegen id : 15] +(45) Project [codegen id : 15] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -(47) HashAggregate [codegen id : 15] +(46) HashAggregate [codegen id : 15] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#32, isEmpty#33] Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] -(48) HashAggregate [codegen id : 15] +(47) HashAggregate [codegen id : 15] Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -(49) Filter [codegen id : 15] +(48) Filter [codegen id : 15] Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) -(50) Project [codegen id : 15] +(49) Project [codegen id : 15] Output [1]: [c_customer_sk#30] Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -(51) Sort [codegen id : 15] +(50) Sort [codegen id : 15] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(52) SortMergeJoin +(51) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#30] Join condition: None -(53) Project [codegen id : 17] +(52) Project [codegen id : 17] Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(54) Scan parquet default.date_dim +(53) Scan parquet default.date_dim Output [3]: [d_date_sk#11, d_year#13, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 16] +(54) ColumnarToRow [codegen id : 16] Input [3]: [d_date_sk#11, d_year#13, d_moy#40] -(56) Filter [codegen id : 16] +(55) Filter [codegen id : 16] Input [3]: [d_date_sk#11, d_year#13, d_moy#40] Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#40)) AND (d_year#13 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#11)) -(57) Project [codegen id : 16] +(56) Project [codegen id : 16] Output [1]: [d_date_sk#11] Input [3]: [d_date_sk#11, d_year#13, d_moy#40] -(58) BroadcastExchange +(57) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] -(59) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#11] Join condition: None -(60) Project [codegen id : 17] +(59) Project [codegen id : 17] Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#11] -(61) Scan parquet default.web_sales +(60) Scan parquet default.web_sales Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(62) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] - -(63) Filter [codegen id : 18] +(61) ColumnarToRow [codegen id : 18] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Condition : (isnotnull(ws_item_sk#43) AND isnotnull(ws_bill_customer_sk#44)) -(64) Exchange +(62) Exchange Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: hashpartitioning(ws_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#48] -(65) Sort [codegen id : 19] +(63) Sort [codegen id : 19] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: [ws_item_sk#43 ASC NULLS FIRST], false, 0 -(66) ReusedExchange [Reuses operator id: unknown] +(64) ReusedExchange [Reuses operator id: unknown] Output [2]: [ss_item_sk#8, d_date#12] -(67) Sort [codegen id : 22] +(65) Sort [codegen id : 22] Input [2]: [ss_item_sk#8, d_date#12] Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 -(68) ReusedExchange [Reuses operator id: 21] +(66) ReusedExchange [Reuses operator id: 20] Output [2]: [i_item_sk#16, i_item_desc#17] -(69) Sort [codegen id : 24] +(67) Sort [codegen id : 24] Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 -(70) SortMergeJoin [codegen id : 25] +(68) SortMergeJoin [codegen id : 25] Left keys [1]: [ss_item_sk#8] Right keys [1]: [i_item_sk#16] Join condition: None -(71) Project [codegen id : 25] +(69) Project [codegen id : 25] Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] -(72) HashAggregate [codegen id : 25] +(70) HashAggregate [codegen id : 25] Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#50] Results [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] -(73) HashAggregate [codegen id : 25] +(71) HashAggregate [codegen id : 25] Input [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] Keys [3]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#52] Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#52 AS count(1)#53] -(74) Filter [codegen id : 25] +(72) Filter [codegen id : 25] Input [2]: [item_sk#23, count(1)#53] Condition : (count(1)#53 > 4) -(75) Project [codegen id : 25] +(73) Project [codegen id : 25] Output [1]: [item_sk#23] Input [2]: [item_sk#23, count(1)#53] -(76) Sort [codegen id : 25] +(74) Sort [codegen id : 25] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 -(77) SortMergeJoin +(75) SortMergeJoin Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#23] Join condition: None -(78) Project [codegen id : 26] +(76) Project [codegen id : 26] Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(79) Exchange +(77) Exchange Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#54] -(80) Sort [codegen id : 27] +(78) Sort [codegen id : 27] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 -(81) ReusedExchange [Reuses operator id: 38] +(79) ReusedExchange [Reuses operator id: 37] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -(82) Sort [codegen id : 29] +(80) Sort [codegen id : 29] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 -(83) ReusedExchange [Reuses operator id: 43] +(81) ReusedExchange [Reuses operator id: 42] Output [1]: [c_customer_sk#30] -(84) Sort [codegen id : 31] +(82) Sort [codegen id : 31] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(85) SortMergeJoin [codegen id : 32] +(83) SortMergeJoin [codegen id : 32] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#30] Join condition: None -(86) Project [codegen id : 32] +(84) Project [codegen id : 32] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -(87) HashAggregate [codegen id : 32] +(85) HashAggregate [codegen id : 32] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#55, isEmpty#56] Results [3]: [c_customer_sk#30, sum#57, isEmpty#58] -(88) HashAggregate [codegen id : 32] +(86) HashAggregate [codegen id : 32] Input [3]: [c_customer_sk#30, sum#57, isEmpty#58] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59] Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] -(89) Filter [codegen id : 32] +(87) Filter [codegen id : 32] Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) -(90) Project [codegen id : 32] +(88) Project [codegen id : 32] Output [1]: [c_customer_sk#30] Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] -(91) Sort [codegen id : 32] +(89) Sort [codegen id : 32] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(92) SortMergeJoin +(90) SortMergeJoin Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#30] Join condition: None -(93) Project [codegen id : 34] +(91) Project [codegen id : 34] Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(94) ReusedExchange [Reuses operator id: 58] +(92) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date_sk#11] -(95) BroadcastHashJoin [codegen id : 34] +(93) BroadcastHashJoin [codegen id : 34] Left keys [1]: [ws_sold_date_sk#47] Right keys [1]: [d_date_sk#11] Join condition: None -(96) Project [codegen id : 34] +(94) Project [codegen id : 34] Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#61] Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#11] -(97) Union +(95) Union -(98) HashAggregate [codegen id : 35] +(96) HashAggregate [codegen id : 35] Input [1]: [sales#42] Keys: [] Functions [1]: [partial_sum(sales#42)] Aggregate Attributes [2]: [sum#62, isEmpty#63] Results [2]: [sum#64, isEmpty#65] -(99) Exchange +(97) Exchange Input [2]: [sum#64, isEmpty#65] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] -(100) HashAggregate [codegen id : 36] +(98) HashAggregate [codegen id : 36] Input [2]: [sum#64, isEmpty#65] Keys: [] Functions [1]: [sum(sales#42)] @@ -557,47 +545,47 @@ Results [1]: [sum(sales#42)#67 AS sum(sales)#68] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (101) +ReusedExchange (99) -(101) ReusedExchange [Reuses operator id: 58] +(99) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date_sk#11] -Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -ReusedExchange (102) +Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 +ReusedExchange (100) -(102) ReusedExchange [Reuses operator id: 13] +(100) ReusedExchange [Reuses operator id: 12] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (126) -+- Exchange (125) - +- * HashAggregate (124) - +- * HashAggregate (123) - +- * HashAggregate (122) - +- * Project (121) - +- * SortMergeJoin Inner (120) - :- * Sort (114) - : +- Exchange (113) - : +- * Project (112) - : +- * BroadcastHashJoin Inner BuildRight (111) - : :- * Filter (105) - : : +- * ColumnarToRow (104) - : : +- Scan parquet default.store_sales (103) - : +- BroadcastExchange (110) - : +- * Project (109) - : +- * Filter (108) - : +- * ColumnarToRow (107) - : +- Scan parquet default.date_dim (106) - +- * Sort (119) - +- Exchange (118) - +- * Filter (117) - +- * ColumnarToRow (116) - +- Scan parquet default.customer (115) - - -(103) Scan parquet default.store_sales +Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (124) ++- Exchange (123) + +- * HashAggregate (122) + +- * HashAggregate (121) + +- * HashAggregate (120) + +- * Project (119) + +- * SortMergeJoin Inner (118) + :- * Sort (112) + : +- Exchange (111) + : +- * Project (110) + : +- * BroadcastHashJoin Inner BuildRight (109) + : :- * Filter (103) + : : +- * ColumnarToRow (102) + : : +- Scan parquet default.store_sales (101) + : +- BroadcastExchange (108) + : +- * Project (107) + : +- * Filter (106) + : +- * ColumnarToRow (105) + : +- Scan parquet default.date_dim (104) + +- * Sort (117) + +- Exchange (116) + +- * Filter (115) + +- * ColumnarToRow (114) + +- Scan parquet default.customer (113) + + +(101) Scan parquet default.store_sales Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -605,124 +593,124 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(104) ColumnarToRow [codegen id : 2] +(102) ColumnarToRow [codegen id : 2] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -(105) Filter [codegen id : 2] +(103) Filter [codegen id : 2] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Condition : isnotnull(ss_customer_sk#26) -(106) Scan parquet default.date_dim +(104) Scan parquet default.date_dim Output [2]: [d_date_sk#11, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(107) ColumnarToRow [codegen id : 1] +(105) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#11, d_year#13] -(108) Filter [codegen id : 1] +(106) Filter [codegen id : 1] Input [2]: [d_date_sk#11, d_year#13] Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) -(109) Project [codegen id : 1] +(107) Project [codegen id : 1] Output [1]: [d_date_sk#11] Input [2]: [d_date_sk#11, d_year#13] -(110) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#70] -(111) BroadcastHashJoin [codegen id : 2] +(109) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join condition: None -(112) Project [codegen id : 2] +(110) Project [codegen id : 2] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9, d_date_sk#11] -(113) Exchange +(111) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#71] -(114) Sort [codegen id : 3] +(112) Sort [codegen id : 3] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 -(115) Scan parquet default.customer +(113) Scan parquet default.customer Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(116) ColumnarToRow [codegen id : 4] +(114) ColumnarToRow [codegen id : 4] Input [1]: [c_customer_sk#30] -(117) Filter [codegen id : 4] +(115) Filter [codegen id : 4] Input [1]: [c_customer_sk#30] Condition : isnotnull(c_customer_sk#30) -(118) Exchange +(116) Exchange Input [1]: [c_customer_sk#30] Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#72] -(119) Sort [codegen id : 5] +(117) Sort [codegen id : 5] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(120) SortMergeJoin [codegen id : 6] +(118) SortMergeJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#30] Join condition: None -(121) Project [codegen id : 6] +(119) Project [codegen id : 6] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -(122) HashAggregate [codegen id : 6] +(120) HashAggregate [codegen id : 6] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#73, isEmpty#74] Results [3]: [c_customer_sk#30, sum#75, isEmpty#76] -(123) HashAggregate [codegen id : 6] +(121) HashAggregate [codegen id : 6] Input [3]: [c_customer_sk#30, sum#75, isEmpty#76] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77] Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77 AS csales#78] -(124) HashAggregate [codegen id : 6] +(122) HashAggregate [codegen id : 6] Input [1]: [csales#78] Keys: [] Functions [1]: [partial_max(csales#78)] Aggregate Attributes [1]: [max#79] Results [1]: [max#80] -(125) Exchange +(123) Exchange Input [1]: [max#80] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] -(126) HashAggregate [codegen id : 7] +(124) HashAggregate [codegen id : 7] Input [1]: [max#80] Keys: [] Functions [1]: [max(csales#78)] Aggregate Attributes [1]: [max(csales#78)#82] Results [1]: [max(csales#78)#82 AS tpcds_cmax#83] -Subquery:4 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#69 -ReusedExchange (127) +Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#69 +ReusedExchange (125) -(127) ReusedExchange [Reuses operator id: 110] +(125) ReusedExchange [Reuses operator id: 108] Output [1]: [d_date_sk#11] -Subquery:5 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt index e48dc68bce924..1e3e8aaa3cb96 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt @@ -25,12 +25,11 @@ WholeStageCodegen (36) InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (1) - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 WholeStageCodegen (8) Sort [item_sk] Project [item_sk] @@ -170,11 +169,10 @@ WholeStageCodegen (36) InputAdapter Exchange [ws_item_sk] #15 WholeStageCodegen (18) - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 WholeStageCodegen (25) Sort [item_sk] Project [item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index e3f996ad72e41..692671c9f287d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -1,80 +1,78 @@ == Physical Plan == -* HashAggregate (76) -+- Exchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (54) - : +- * BroadcastHashJoin Inner BuildRight (53) - : :- * Project (47) - : : +- SortMergeJoin LeftSemi (46) - : : :- * Sort (29) - : : : +- Exchange (28) - : : : +- * Project (27) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.catalog_sales (1) - : : : +- BroadcastExchange (25) - : : : +- * Project (24) - : : : +- * Filter (23) - : : : +- * HashAggregate (22) - : : : +- Exchange (21) - : : : +- * HashAggregate (20) - : : : +- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (17) - : : : +- * Filter (16) - : : : +- * ColumnarToRow (15) - : : : +- Scan parquet default.item (14) - : : +- * Sort (45) - : : +- * Project (44) - : : +- * Filter (43) - : : +- * HashAggregate (42) - : : +- Exchange (41) - : : +- * HashAggregate (40) - : : +- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Project (33) - : : : +- * Filter (32) - : : : +- * ColumnarToRow (31) - : : : +- Scan parquet default.store_sales (30) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * ColumnarToRow (35) - : : +- Scan parquet default.customer (34) - : +- BroadcastExchange (52) - : +- * Project (51) - : +- * Filter (50) - : +- * ColumnarToRow (49) - : +- Scan parquet default.date_dim (48) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (69) - : +- SortMergeJoin LeftSemi (68) - : :- * Sort (62) - : : +- Exchange (61) - : : +- * Project (60) - : : +- * BroadcastHashJoin LeftSemi BuildRight (59) - : : :- * Filter (57) - : : : +- * ColumnarToRow (56) - : : : +- Scan parquet default.web_sales (55) - : : +- ReusedExchange (58) - : +- * Sort (67) - : +- * Project (66) - : +- * Filter (65) - : +- * HashAggregate (64) - : +- ReusedExchange (63) - +- ReusedExchange (70) +* HashAggregate (74) ++- Exchange (73) + +- * HashAggregate (72) + +- Union (71) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * Project (46) + : : +- SortMergeJoin LeftSemi (45) + : : :- * Sort (28) + : : : +- Exchange (27) + : : : +- * Project (26) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (25) + : : : :- * ColumnarToRow (2) + : : : : +- Scan parquet default.catalog_sales (1) + : : : +- BroadcastExchange (24) + : : : +- * Project (23) + : : : +- * Filter (22) + : : : +- * HashAggregate (21) + : : : +- Exchange (20) + : : : +- * HashAggregate (19) + : : : +- * Project (18) + : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : :- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * Filter (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- Scan parquet default.store_sales (3) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.date_dim (6) + : : : +- BroadcastExchange (16) + : : : +- * Filter (15) + : : : +- * ColumnarToRow (14) + : : : +- Scan parquet default.item (13) + : : +- * Sort (44) + : : +- * Project (43) + : : +- * Filter (42) + : : +- * HashAggregate (41) + : : +- Exchange (40) + : : +- * HashAggregate (39) + : : +- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Project (32) + : : : +- * Filter (31) + : : : +- * ColumnarToRow (30) + : : : +- Scan parquet default.store_sales (29) + : : +- BroadcastExchange (36) + : : +- * Filter (35) + : : +- * ColumnarToRow (34) + : : +- Scan parquet default.customer (33) + : +- BroadcastExchange (51) + : +- * Project (50) + : +- * Filter (49) + : +- * ColumnarToRow (48) + : +- Scan parquet default.date_dim (47) + +- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (67) + : +- SortMergeJoin LeftSemi (66) + : :- * Sort (60) + : : +- Exchange (59) + : : +- * Project (58) + : : +- * BroadcastHashJoin LeftSemi BuildRight (57) + : : :- * ColumnarToRow (55) + : : : +- Scan parquet default.web_sales (54) + : : +- ReusedExchange (56) + : +- * Sort (65) + : +- * Project (64) + : +- * Filter (63) + : +- * HashAggregate (62) + : +- ReusedExchange (61) + +- ReusedExchange (68) (1) Scan parquet default.catalog_sales @@ -82,17 +80,12 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(3) Filter [codegen id : 5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) - -(4) Scan parquet default.store_sales +(3) Scan parquet default.store_sales Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -100,325 +93,320 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 3] +(4) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(6) Filter [codegen id : 3] +(5) Filter [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_date#11, d_year#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#12] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#12] Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [2]: [d_date_sk#10, d_date#11] Input [3]: [d_date_sk#10, d_date#11, d_year#12] -(11) BroadcastExchange +(10) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(12) BroadcastHashJoin [codegen id : 3] +(11) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(13) Project [codegen id : 3] +(12) Project [codegen id : 3] Output [2]: [ss_item_sk#7, d_date#11] Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -(14) Scan parquet default.item +(13) Scan parquet default.item Output [2]: [i_item_sk#14, i_item_desc#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 2] +(14) ColumnarToRow [codegen id : 2] Input [2]: [i_item_sk#14, i_item_desc#15] -(16) Filter [codegen id : 2] +(15) Filter [codegen id : 2] Input [2]: [i_item_sk#14, i_item_desc#15] Condition : isnotnull(i_item_sk#14) -(17) BroadcastExchange +(16) BroadcastExchange Input [2]: [i_item_sk#14, i_item_desc#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] -(18) BroadcastHashJoin [codegen id : 3] +(17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#7] Right keys [1]: [i_item_sk#14] Join condition: None -(19) Project [codegen id : 3] +(18) Project [codegen id : 3] Output [3]: [d_date#11, i_item_sk#14, i_item_desc#15] Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#14, i_item_desc#15] -(20) HashAggregate [codegen id : 3] +(19) HashAggregate [codegen id : 3] Input [3]: [d_date#11, i_item_sk#14, i_item_desc#15] Keys [3]: [substr(i_item_desc#15, 1, 30) AS substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#18] Results [4]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11, count#19] -(21) Exchange +(20) Exchange Input [4]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11, count#19] Arguments: hashpartitioning(substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11, 5), ENSURE_REQUIREMENTS, [id=#20] -(22) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 4] Input [4]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11, count#19] Keys [3]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#21] Results [2]: [i_item_sk#14 AS item_sk#22, count(1)#21 AS count(1)#23] -(23) Filter [codegen id : 4] +(22) Filter [codegen id : 4] Input [2]: [item_sk#22, count(1)#23] Condition : (count(1)#23 > 4) -(24) Project [codegen id : 4] +(23) Project [codegen id : 4] Output [1]: [item_sk#22] Input [2]: [item_sk#22, count(1)#23] -(25) BroadcastExchange +(24) BroadcastExchange Input [1]: [item_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(26) BroadcastHashJoin [codegen id : 5] +(25) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#22] Join condition: None -(27) Project [codegen id : 5] +(26) Project [codegen id : 5] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(28) Exchange +(27) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#25] -(29) Sort [codegen id : 6] +(28) Sort [codegen id : 6] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(30) Scan parquet default.store_sales +(29) Scan parquet default.store_sales Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 8] +(30) ColumnarToRow [codegen id : 8] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -(32) Filter [codegen id : 8] +(31) Filter [codegen id : 8] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Condition : isnotnull(ss_customer_sk#26) -(33) Project [codegen id : 8] +(32) Project [codegen id : 8] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -(34) Scan parquet default.customer +(33) Scan parquet default.customer Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 7] +(34) ColumnarToRow [codegen id : 7] Input [1]: [c_customer_sk#29] -(36) Filter [codegen id : 7] +(35) Filter [codegen id : 7] Input [1]: [c_customer_sk#29] Condition : isnotnull(c_customer_sk#29) -(37) BroadcastExchange +(36) BroadcastExchange Input [1]: [c_customer_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] -(38) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#29] Join condition: None -(39) Project [codegen id : 8] +(38) Project [codegen id : 8] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -(40) HashAggregate [codegen id : 8] +(39) HashAggregate [codegen id : 8] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] Keys [1]: [c_customer_sk#29] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#31, isEmpty#32] Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] -(41) Exchange +(40) Exchange Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#35] -(42) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 9] Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -(43) Filter [codegen id : 9] +(42) Filter [codegen id : 9] Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) -(44) Project [codegen id : 9] +(43) Project [codegen id : 9] Output [1]: [c_customer_sk#29] Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -(45) Sort [codegen id : 9] +(44) Sort [codegen id : 9] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(46) SortMergeJoin +(45) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#29] Join condition: None -(47) Project [codegen id : 11] +(46) Project [codegen id : 11] Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(48) Scan parquet default.date_dim +(47) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#12, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 10] +(48) ColumnarToRow [codegen id : 10] Input [3]: [d_date_sk#10, d_year#12, d_moy#40] -(50) Filter [codegen id : 10] +(49) Filter [codegen id : 10] Input [3]: [d_date_sk#10, d_year#12, d_moy#40] Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#40)) AND (d_year#12 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#10)) -(51) Project [codegen id : 10] +(50) Project [codegen id : 10] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#12, d_moy#40] -(52) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] -(53) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#10] Join condition: None -(54) Project [codegen id : 11] +(53) Project [codegen id : 11] Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#10] -(55) Scan parquet default.web_sales +(54) Scan parquet default.web_sales Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 16] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] - -(57) Filter [codegen id : 16] +(55) ColumnarToRow [codegen id : 16] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Condition : (isnotnull(ws_item_sk#43) AND isnotnull(ws_bill_customer_sk#44)) -(58) ReusedExchange [Reuses operator id: unknown] +(56) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] -(59) BroadcastHashJoin [codegen id : 16] +(57) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#22] Join condition: None -(60) Project [codegen id : 16] +(58) Project [codegen id : 16] Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(61) Exchange +(59) Exchange Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#48] -(62) Sort [codegen id : 17] +(60) Sort [codegen id : 17] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 -(63) ReusedExchange [Reuses operator id: 41] +(61) ReusedExchange [Reuses operator id: 40] Output [3]: [c_customer_sk#29, sum#49, isEmpty#50] -(64) HashAggregate [codegen id : 20] +(62) HashAggregate [codegen id : 20] Input [3]: [c_customer_sk#29, sum#49, isEmpty#50] Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51] Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] -(65) Filter [codegen id : 20] +(63) Filter [codegen id : 20] Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) -(66) Project [codegen id : 20] +(64) Project [codegen id : 20] Output [1]: [c_customer_sk#29] Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] -(67) Sort [codegen id : 20] +(65) Sort [codegen id : 20] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(68) SortMergeJoin +(66) SortMergeJoin Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#29] Join condition: None -(69) Project [codegen id : 22] +(67) Project [codegen id : 22] Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(70) ReusedExchange [Reuses operator id: 52] +(68) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#10] -(71) BroadcastHashJoin [codegen id : 22] +(69) BroadcastHashJoin [codegen id : 22] Left keys [1]: [ws_sold_date_sk#47] Right keys [1]: [d_date_sk#10] Join condition: None -(72) Project [codegen id : 22] +(70) Project [codegen id : 22] Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#53] Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#10] -(73) Union +(71) Union -(74) HashAggregate [codegen id : 23] +(72) HashAggregate [codegen id : 23] Input [1]: [sales#42] Keys: [] Functions [1]: [partial_sum(sales#42)] Aggregate Attributes [2]: [sum#54, isEmpty#55] Results [2]: [sum#56, isEmpty#57] -(75) Exchange +(73) Exchange Input [2]: [sum#56, isEmpty#57] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#58] -(76) HashAggregate [codegen id : 24] +(74) HashAggregate [codegen id : 24] Input [2]: [sum#56, isEmpty#57] Keys: [] Functions [1]: [sum(sales#42)] @@ -428,45 +416,45 @@ Results [1]: [sum(sales#42)#59 AS sum(sales)#60] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (77) +ReusedExchange (75) -(77) ReusedExchange [Reuses operator id: 52] +(75) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (78) +Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +ReusedExchange (76) -(78) ReusedExchange [Reuses operator id: 11] +(76) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (100) -+- Exchange (99) - +- * HashAggregate (98) - +- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- * Project (94) - +- * BroadcastHashJoin Inner BuildRight (93) - :- * Project (87) - : +- * BroadcastHashJoin Inner BuildRight (86) - : :- * Filter (81) - : : +- * ColumnarToRow (80) - : : +- Scan parquet default.store_sales (79) - : +- BroadcastExchange (85) - : +- * Filter (84) - : +- * ColumnarToRow (83) - : +- Scan parquet default.customer (82) - +- BroadcastExchange (92) - +- * Project (91) - +- * Filter (90) - +- * ColumnarToRow (89) - +- Scan parquet default.date_dim (88) - - -(79) Scan parquet default.store_sales +Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (98) ++- Exchange (97) + +- * HashAggregate (96) + +- * HashAggregate (95) + +- Exchange (94) + +- * HashAggregate (93) + +- * Project (92) + +- * BroadcastHashJoin Inner BuildRight (91) + :- * Project (85) + : +- * BroadcastHashJoin Inner BuildRight (84) + : :- * Filter (79) + : : +- * ColumnarToRow (78) + : : +- Scan parquet default.store_sales (77) + : +- BroadcastExchange (83) + : +- * Filter (82) + : +- * ColumnarToRow (81) + : +- Scan parquet default.customer (80) + +- BroadcastExchange (90) + +- * Project (89) + +- * Filter (88) + +- * ColumnarToRow (87) + +- Scan parquet default.date_dim (86) + + +(77) Scan parquet default.store_sales Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -474,116 +462,116 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 3] +(78) ColumnarToRow [codegen id : 3] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -(81) Filter [codegen id : 3] +(79) Filter [codegen id : 3] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Condition : isnotnull(ss_customer_sk#26) -(82) Scan parquet default.customer +(80) Scan parquet default.customer Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(83) ColumnarToRow [codegen id : 1] +(81) ColumnarToRow [codegen id : 1] Input [1]: [c_customer_sk#29] -(84) Filter [codegen id : 1] +(82) Filter [codegen id : 1] Input [1]: [c_customer_sk#29] Condition : isnotnull(c_customer_sk#29) -(85) BroadcastExchange +(83) BroadcastExchange Input [1]: [c_customer_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#62] -(86) BroadcastHashJoin [codegen id : 3] +(84) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#29] Join condition: None -(87) Project [codegen id : 3] +(85) Project [codegen id : 3] Output [4]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] -(88) Scan parquet default.date_dim +(86) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 2] +(87) ColumnarToRow [codegen id : 2] Input [2]: [d_date_sk#10, d_year#12] -(90) Filter [codegen id : 2] +(88) Filter [codegen id : 2] Input [2]: [d_date_sk#10, d_year#12] Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(91) Project [codegen id : 2] +(89) Project [codegen id : 2] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#12] -(92) BroadcastExchange +(90) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] -(93) BroadcastHashJoin [codegen id : 3] +(91) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(94) Project [codegen id : 3] +(92) Project [codegen id : 3] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] Input [5]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29, d_date_sk#10] -(95) HashAggregate [codegen id : 3] +(93) HashAggregate [codegen id : 3] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] Keys [1]: [c_customer_sk#29] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#64, isEmpty#65] Results [3]: [c_customer_sk#29, sum#66, isEmpty#67] -(96) Exchange +(94) Exchange Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#68] -(97) HashAggregate [codegen id : 4] +(95) HashAggregate [codegen id : 4] Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS csales#70] -(98) HashAggregate [codegen id : 4] +(96) HashAggregate [codegen id : 4] Input [1]: [csales#70] Keys: [] Functions [1]: [partial_max(csales#70)] Aggregate Attributes [1]: [max#71] Results [1]: [max#72] -(99) Exchange +(97) Exchange Input [1]: [max#72] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#73] -(100) HashAggregate [codegen id : 5] +(98) HashAggregate [codegen id : 5] Input [1]: [max#72] Keys: [] Functions [1]: [max(csales#70)] Aggregate Attributes [1]: [max(csales#70)#74] Results [1]: [max(csales#70)#74 AS tpcds_cmax#75] -Subquery:4 Hosting operator id = 79 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#61 -ReusedExchange (101) +Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#61 +ReusedExchange (99) -(101) ReusedExchange [Reuses operator id: 92] +(99) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#10] -Subquery:5 Hosting operator id = 55 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index 36bc26bc541bc..87f135b3fff0d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -19,12 +19,11 @@ WholeStageCodegen (24) WholeStageCodegen (5) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 WholeStageCodegen (4) @@ -141,11 +140,10 @@ WholeStageCodegen (24) WholeStageCodegen (16) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [item_sk] #15 WholeStageCodegen (20) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt index 0ac39b5242dce..35a42fa256325 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt @@ -143,7 +143,7 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -151,7 +151,7 @@ Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, (3) Filter [codegen id : 1] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_bill_customer_sk#1) AND isnotnull(cs_item_sk#2)) +Condition : isnotnull(cs_bill_customer_sk#1) (4) Exchange Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] @@ -516,7 +516,7 @@ Output [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_pric Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (83) ColumnarToRow [codegen id : 27] @@ -524,7 +524,7 @@ Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price (84) Filter [codegen id : 27] Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Condition : (isnotnull(ws_bill_customer_sk#54) AND isnotnull(ws_item_sk#53)) +Condition : isnotnull(ws_bill_customer_sk#54) (85) Exchange Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt index 110e3e50157ba..1962a53fa34c2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt @@ -27,7 +27,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (1) - Filter [cs_bill_customer_sk,cs_item_sk] + Filter [cs_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] @@ -210,7 +210,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter Exchange [ws_item_sk] #17 WholeStageCodegen (27) - Filter [ws_bill_customer_sk,ws_item_sk] + Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 7b4a41ff0813f..385cf017e58c3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -109,7 +109,7 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 5] @@ -117,7 +117,7 @@ Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, (3) Filter [codegen id : 5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_bill_customer_sk#1) AND isnotnull(cs_item_sk#2)) +Condition : isnotnull(cs_bill_customer_sk#1) (4) Scan parquet default.store_sales Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] @@ -435,7 +435,7 @@ Output [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_pric Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 22] @@ -443,7 +443,7 @@ Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price (73) Filter [codegen id : 22] Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Condition : (isnotnull(ws_bill_customer_sk#55) AND isnotnull(ws_item_sk#54)) +Condition : isnotnull(ws_bill_customer_sk#55) (74) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index e405cfaa0d9bf..a18328434b618 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_bill_customer_sk,cs_item_sk] + Filter [cs_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] @@ -167,7 +167,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (22) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_bill_customer_sk,ws_item_sk] + Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index aff4892e0231e..1349aa54f3cc2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_manufact_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_manufact_id#14)) +Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics ), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : ((isnotnull(i_category#15) AND (i_category#15 = Electronics )) AND isnotnull(i_manufact_id#14)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index 6a4f8b5362c74..d1a115fa528d5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk,i_manufact_id] + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #6 WholeStageCodegen (3) Project [i_manufact_id] - Filter [i_category,i_manufact_id] + Filter [i_category] ColumnarToRow InputAdapter Scan parquet default.item [i_category,i_manufact_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index aff4892e0231e..1349aa54f3cc2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_manufact_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_manufact_id#14)) +Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics ), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : ((isnotnull(i_category#15) AND (i_category#15 = Electronics )) AND isnotnull(i_manufact_id#14)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index 6a4f8b5362c74..d1a115fa528d5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk,i_manufact_id] + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #6 WholeStageCodegen (3) Project [i_manufact_id] - Filter [i_category,i_manufact_id] + Filter [i_category] ColumnarToRow InputAdapter Scan parquet default.item [i_category,i_manufact_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index 82127815c5d69..7098389df6776 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -1,69 +1,68 @@ == Physical Plan == -TakeOrderedAndProject (58) -+- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * SortMergeJoin Inner (53) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (38) - : : +- Exchange (37) - : : +- * Project (36) - : : +- * Filter (35) - : : +- SortMergeJoin ExistenceJoin(exists#1) (34) - : : :- SortMergeJoin ExistenceJoin(exists#2) (26) - : : : :- SortMergeJoin LeftSemi (18) +TakeOrderedAndProject (57) ++- * HashAggregate (56) + +- Exchange (55) + +- * HashAggregate (54) + +- * Project (53) + +- * SortMergeJoin Inner (52) + :- * Sort (46) + : +- Exchange (45) + : +- * Project (44) + : +- * SortMergeJoin Inner (43) + : :- * Sort (37) + : : +- Exchange (36) + : : +- * Project (35) + : : +- * Filter (34) + : : +- SortMergeJoin ExistenceJoin(exists#1) (33) + : : :- SortMergeJoin ExistenceJoin(exists#2) (25) + : : : :- SortMergeJoin LeftSemi (17) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- * Sort (17) - : : : : +- Exchange (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Filter (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- Scan parquet default.store_sales (6) - : : : : +- BroadcastExchange (13) - : : : : +- * Project (12) - : : : : +- * Filter (11) - : : : : +- * ColumnarToRow (10) - : : : : +- Scan parquet default.date_dim (9) - : : : +- * Sort (25) - : : : +- Exchange (24) - : : : +- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * ColumnarToRow (20) - : : : : +- Scan parquet default.web_sales (19) - : : : +- ReusedExchange (21) - : : +- * Sort (33) - : : +- Exchange (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- Scan parquet default.catalog_sales (27) - : : +- ReusedExchange (29) - : +- * Sort (43) - : +- Exchange (42) - : +- * Filter (41) - : +- * ColumnarToRow (40) - : +- Scan parquet default.customer_address (39) - +- * Sort (52) - +- Exchange (51) - +- * Filter (50) - +- * ColumnarToRow (49) - +- Scan parquet default.customer_demographics (48) + : : : : +- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Project (14) + : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : :- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (12) + : : : : +- * Project (11) + : : : : +- * Filter (10) + : : : : +- * ColumnarToRow (9) + : : : : +- Scan parquet default.date_dim (8) + : : : +- * Sort (24) + : : : +- Exchange (23) + : : : +- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- Scan parquet default.web_sales (18) + : : : +- ReusedExchange (20) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * ColumnarToRow (27) + : : : +- Scan parquet default.catalog_sales (26) + : : +- ReusedExchange (28) + : +- * Sort (42) + : +- Exchange (41) + : +- * Filter (40) + : +- * ColumnarToRow (39) + : +- Scan parquet default.customer_address (38) + +- * Sort (51) + +- Exchange (50) + +- * Filter (49) + +- * ColumnarToRow (48) + +- Scan parquet default.customer_demographics (47) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -71,7 +70,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -86,249 +85,244 @@ Output [2]: [ss_customer_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -(8) Filter [codegen id : 4] -Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_customer_sk#7) - -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_qoy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_qoy#12)) AND (d_year#11 = 2002)) AND (d_qoy#12 < 4)) AND isnotnull(d_date_sk#10)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] -(13) BroadcastExchange +(12) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [1]: [ss_customer_sk#7] Input [3]: [ss_customer_sk#7, ss_sold_date_sk#8, d_date_sk#10] -(16) Exchange +(15) Exchange Input [1]: [ss_customer_sk#7] Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [1]: [ss_customer_sk#7] Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(18) SortMergeJoin +(17) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#7] Join condition: None -(19) Scan parquet default.web_sales +(18) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#16), dynamicpruningexpression(ws_sold_date_sk#16 IN dynamicpruning#9)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] -(21) ReusedExchange [Reuses operator id: 13] +(20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -(22) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(23) Project [codegen id : 7] +(22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] -(24) Exchange +(23) Exchange Input [1]: [ws_bill_customer_sk#15] Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] -(25) Sort [codegen id : 8] +(24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] Arguments: [ws_bill_customer_sk#15 ASC NULLS FIRST], false, 0 -(26) SortMergeJoin +(25) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#15] Join condition: None -(27) Scan parquet default.catalog_sales +(26) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 10] +(27) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] -(29) ReusedExchange [Reuses operator id: 13] +(28) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -(30) BroadcastHashJoin [codegen id : 10] +(29) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#19] Right keys [1]: [d_date_sk#10] Join condition: None -(31) Project [codegen id : 10] +(30) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#18] Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] -(32) Exchange +(31) Exchange Input [1]: [cs_ship_customer_sk#18] Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] -(33) Sort [codegen id : 11] +(32) Sort [codegen id : 11] Input [1]: [cs_ship_customer_sk#18] Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#18] Join condition: None -(35) Filter [codegen id : 12] +(34) Filter [codegen id : 12] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(36) Project [codegen id : 12] +(35) Project [codegen id : 12] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(37) Exchange +(36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] -(38) Sort [codegen id : 13] +(37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 -(39) Scan parquet default.customer_address +(38) Scan parquet default.customer_address Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(40) ColumnarToRow [codegen id : 14] +(39) ColumnarToRow [codegen id : 14] Input [2]: [ca_address_sk#22, ca_state#23] -(41) Filter [codegen id : 14] +(40) Filter [codegen id : 14] Input [2]: [ca_address_sk#22, ca_state#23] Condition : isnotnull(ca_address_sk#22) -(42) Exchange +(41) Exchange Input [2]: [ca_address_sk#22, ca_state#23] Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] -(43) Sort [codegen id : 15] +(42) Sort [codegen id : 15] Input [2]: [ca_address_sk#22, ca_state#23] Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 -(44) SortMergeJoin [codegen id : 16] +(43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#22] Join condition: None -(45) Project [codegen id : 16] +(44) Project [codegen id : 16] Output [2]: [c_current_cdemo_sk#4, ca_state#23] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] -(46) Exchange +(45) Exchange Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] -(47) Sort [codegen id : 17] +(46) Sort [codegen id : 17] Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(48) Scan parquet default.customer_demographics +(47) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 18] +(48) ColumnarToRow [codegen id : 18] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(50) Filter [codegen id : 18] +(49) Filter [codegen id : 18] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Condition : isnotnull(cd_demo_sk#26) -(51) Exchange +(50) Exchange Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] -(52) Sort [codegen id : 19] +(51) Sort [codegen id : 19] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 -(53) SortMergeJoin [codegen id : 20] +(52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#26] Join condition: None -(54) Project [codegen id : 20] +(53) Project [codegen id : 20] Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(55) HashAggregate [codegen id : 20] +(54) HashAggregate [codegen id : 20] Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [10]: [partial_count(1), partial_min(cd_dep_count#29), partial_max(cd_dep_count#29), partial_avg(cd_dep_count#29), partial_min(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_avg(cd_dep_employed_count#30), partial_min(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_avg(cd_dep_college_count#31)] Aggregate Attributes [13]: [count#33, min#34, max#35, sum#36, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45] Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] -(56) Exchange +(55) Exchange Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] -(57) HashAggregate [codegen id : 21] +(56) HashAggregate [codegen id : 21] Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [10]: [count(1), min(cd_dep_count#29), max(cd_dep_count#29), avg(cd_dep_count#29), min(cd_dep_employed_count#30), max(cd_dep_employed_count#30), avg(cd_dep_employed_count#30), min(cd_dep_college_count#31), max(cd_dep_college_count#31), avg(cd_dep_college_count#31)] Aggregate Attributes [10]: [count(1)#60, min(cd_dep_count#29)#61, max(cd_dep_count#29)#62, avg(cd_dep_count#29)#63, min(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, avg(cd_dep_employed_count#30)#66, min(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, avg(cd_dep_college_count#31)#69] Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, count(1)#60 AS cnt1#70, min(cd_dep_count#29)#61 AS min(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, avg(cd_dep_count#29)#63 AS avg(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, min(cd_dep_employed_count#30)#64 AS min(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, avg(cd_dep_employed_count#30)#66 AS avg(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, min(cd_dep_college_count#31)#67 AS min(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, avg(cd_dep_college_count#31)#69 AS avg(cd_dep_college_count)#81, cd_dep_count#29 AS aggOrder#82] -(58) TakeOrderedAndProject +(57) TakeOrderedAndProject Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81, aggOrder#82] Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, aggOrder#82 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (59) +ReusedExchange (58) -(59) ReusedExchange [Reuses operator id: 13] +(58) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt index 60624e1380e63..6c399f1bd2197 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -43,12 +43,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #6 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #6 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 661cac7ce3ef0..4dab13491784a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -1,58 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (31) - : : +- * Filter (30) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (29) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (22) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Project (30) + : : +- * Filter (29) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (28) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (21) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (14) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (14) - : : : : +- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (21) - : : : +- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (18) - : : +- BroadcastExchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * ColumnarToRow (24) - : : : +- Scan parquet default.catalog_sales (23) - : : +- ReusedExchange (25) - : +- BroadcastExchange (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.customer_address (32) - +- BroadcastExchange (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.customer_demographics (38) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * ColumnarToRow (5) + : : : : : +- Scan parquet default.store_sales (4) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.date_dim (6) + : : : +- BroadcastExchange (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- BroadcastExchange (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (24) + : +- BroadcastExchange (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.customer_address (31) + +- BroadcastExchange (40) + +- * Filter (39) + +- * ColumnarToRow (38) + +- Scan parquet default.customer_demographics (37) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -60,220 +59,215 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) Filter [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_customer_sk#6) - -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -(11) BroadcastExchange +(10) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] -(12) BroadcastHashJoin [codegen id : 2] +(11) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join condition: None -(13) Project [codegen id : 2] +(12) Project [codegen id : 2] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(14) BroadcastExchange +(13) BroadcastExchange Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(15) BroadcastHashJoin [codegen id : 9] +(14) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join condition: None -(16) Scan parquet default.web_sales +(15) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#15), dynamicpruningexpression(ws_sold_date_sk#15 IN dynamicpruning#8)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] +(16) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] -(18) ReusedExchange [Reuses operator id: 11] +(17) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -(19) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] Right keys [1]: [d_date_sk#9] Join condition: None -(20) Project [codegen id : 4] +(19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] -(21) BroadcastExchange +(20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(22) BroadcastHashJoin [codegen id : 9] +(21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#14] Join condition: None -(23) Scan parquet default.catalog_sales +(22) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 6] +(23) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] -(25) ReusedExchange [Reuses operator id: 11] +(24) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -(26) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#9] Join condition: None -(27) Project [codegen id : 6] +(26) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#17] Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] -(28) BroadcastExchange +(27) BroadcastExchange Input [1]: [cs_ship_customer_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(29) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#17] Join condition: None -(30) Filter [codegen id : 9] +(29) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(31) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(32) Scan parquet default.customer_address +(31) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 7] +(32) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#20, ca_state#21] -(34) Filter [codegen id : 7] +(33) Filter [codegen id : 7] Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(35) BroadcastExchange +(34) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(36) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join condition: None -(37) Project [codegen id : 9] +(36) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, ca_state#21] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] -(38) Scan parquet default.customer_demographics +(37) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 8] +(38) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(40) Filter [codegen id : 8] +(39) Filter [codegen id : 8] Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Condition : isnotnull(cd_demo_sk#23) -(41) BroadcastExchange +(40) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(42) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join condition: None -(43) Project [codegen id : 9] +(42) Project [codegen id : 9] Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(44) HashAggregate [codegen id : 9] +(43) HashAggregate [codegen id : 9] Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] Aggregate Attributes [13]: [count#30, min#31, max#32, sum#33, count#34, min#35, max#36, sum#37, count#38, min#39, max#40, sum#41, count#42] Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] -(45) Exchange +(44) Exchange Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] -(46) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 10] Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] Aggregate Attributes [10]: [count(1)#57, min(cd_dep_count#26)#58, max(cd_dep_count#26)#59, avg(cd_dep_count#26)#60, min(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, avg(cd_dep_employed_count#27)#63, min(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, avg(cd_dep_college_count#28)#66] Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, count(1)#57 AS cnt1#67, min(cd_dep_count#26)#58 AS min(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, avg(cd_dep_count#26)#60 AS avg(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, min(cd_dep_employed_count#27)#61 AS min(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, avg(cd_dep_employed_count#27)#63 AS avg(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, min(cd_dep_college_count#28)#64 AS min(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, avg(cd_dep_college_count#28)#66 AS avg(cd_dep_college_count)#78, cd_dep_count#26 AS aggOrder#79] -(47) TakeOrderedAndProject +(46) TakeOrderedAndProject Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78, aggOrder#79] Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, aggOrder#79 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -ReusedExchange (48) +ReusedExchange (47) -(48) ReusedExchange [Reuses operator id: 11] +(47) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index 4010c2656300c..2614f4f8ae881 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -23,12 +23,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index 30ecc62144ddb..a754cef8f2962 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (82) -+- * HashAggregate (81) - +- Exchange (80) - +- * HashAggregate (79) - +- * Expand (78) - +- Union (77) +TakeOrderedAndProject (81) ++- * HashAggregate (80) + +- Exchange (79) + +- * HashAggregate (78) + +- * Expand (77) + +- Union (76) :- * HashAggregate (25) : +- Exchange (24) : +- * HashAggregate (23) @@ -51,36 +51,35 @@ TakeOrderedAndProject (82) : : +- * ColumnarToRow (36) : : +- Scan parquet default.catalog_page (35) : +- ReusedExchange (41) - +- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- * Project (73) - +- * BroadcastHashJoin Inner BuildRight (72) - :- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- Union (64) + +- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- * Project (69) + : +- * BroadcastHashJoin Inner BuildRight (68) + : :- Union (63) : : :- * Project (50) : : : +- * Filter (49) : : : +- * ColumnarToRow (48) : : : +- Scan parquet default.web_sales (47) - : : +- * Project (63) - : : +- * SortMergeJoin Inner (62) - : : :- * Sort (55) - : : : +- Exchange (54) - : : : +- * Filter (53) - : : : +- * ColumnarToRow (52) - : : : +- Scan parquet default.web_returns (51) - : : +- * Sort (61) - : : +- Exchange (60) - : : +- * Project (59) - : : +- * Filter (58) - : : +- * ColumnarToRow (57) - : : +- Scan parquet default.web_sales (56) - : +- BroadcastExchange (68) - : +- * Filter (67) - : +- * ColumnarToRow (66) - : +- Scan parquet default.web_site (65) - +- ReusedExchange (71) + : : +- * Project (62) + : : +- * SortMergeJoin Inner (61) + : : :- * Sort (54) + : : : +- Exchange (53) + : : : +- * ColumnarToRow (52) + : : : +- Scan parquet default.web_returns (51) + : : +- * Sort (60) + : : +- Exchange (59) + : : +- * Project (58) + : : +- * Filter (57) + : : +- * ColumnarToRow (56) + : : +- Scan parquet default.web_sales (55) + : +- BroadcastExchange (67) + : +- * Filter (66) + : +- * ColumnarToRow (65) + : +- Scan parquet default.web_site (64) + +- ReusedExchange (70) (1) Scan parquet default.store_sales @@ -319,162 +318,157 @@ Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#1 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -(53) Filter [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Condition : (isnotnull(wr_item_sk#98) AND isnotnull(wr_order_number#99)) - -(54) Exchange +(53) Exchange Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] -(55) Sort [codegen id : 15] +(54) Sort [codegen id : 15] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 -(56) Scan parquet default.web_sales +(55) Scan parquet default.web_sales Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(57) ColumnarToRow [codegen id : 16] +(56) ColumnarToRow [codegen id : 16] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(58) Filter [codegen id : 16] +(57) Filter [codegen id : 16] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) -(59) Project [codegen id : 16] +(58) Project [codegen id : 16] Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(60) Exchange +(59) Exchange Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] -(61) Sort [codegen id : 17] +(60) Sort [codegen id : 17] Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 -(62) SortMergeJoin [codegen id : 18] +(61) SortMergeJoin [codegen id : 18] Left keys [2]: [wr_item_sk#98, wr_order_number#99] Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None -(63) Project [codegen id : 18] +(62) Project [codegen id : 18] Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -(64) Union +(63) Union -(65) Scan parquet default.web_site +(64) Scan parquet default.web_site Output [2]: [web_site_sk#113, web_site_id#114] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] +(65) ColumnarToRow [codegen id : 19] Input [2]: [web_site_sk#113, web_site_id#114] -(67) Filter [codegen id : 19] +(66) Filter [codegen id : 19] Input [2]: [web_site_sk#113, web_site_id#114] Condition : isnotnull(web_site_sk#113) -(68) BroadcastExchange +(67) BroadcastExchange Input [2]: [web_site_sk#113, web_site_id#114] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] -(69) BroadcastHashJoin [codegen id : 21] +(68) BroadcastHashJoin [codegen id : 21] Left keys [1]: [wsr_web_site_sk#92] Right keys [1]: [web_site_sk#113] Join condition: None -(70) Project [codegen id : 21] +(69) Project [codegen id : 21] Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] -(71) ReusedExchange [Reuses operator id: 20] +(70) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] -(72) BroadcastHashJoin [codegen id : 21] +(71) BroadcastHashJoin [codegen id : 21] Left keys [1]: [date_sk#93] Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None -(73) Project [codegen id : 21] +(72) Project [codegen id : 21] Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] -(74) HashAggregate [codegen id : 21] +(73) HashAggregate [codegen id : 21] Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Keys [1]: [web_site_id#114] Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -(75) Exchange +(74) Exchange Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] -(76) HashAggregate [codegen id : 22] +(75) HashAggregate [codegen id : 22] Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] Keys [1]: [web_site_id#114] Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#129, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS RETURNS#130, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#131, web channel AS channel#132, concat(web_site, web_site_id#114) AS id#133] -(77) Union +(76) Union -(78) Expand [codegen id : 23] +(77) Expand [codegen id : 23] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] -(79) HashAggregate [codegen id : 23] +(78) HashAggregate [codegen id : 23] Input [6]: [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] Keys [3]: [channel#134, id#135, spark_grouping_id#136] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] Results [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -(80) Exchange +(79) Exchange Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Arguments: hashpartitioning(channel#134, id#135, spark_grouping_id#136, 5), ENSURE_REQUIREMENTS, [id=#149] -(81) HashAggregate [codegen id : 24] +(80) HashAggregate [codegen id : 24] Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Keys [3]: [channel#134, id#135, spark_grouping_id#136] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] Aggregate Attributes [3]: [sum(sales#41)#150, sum(returns#42)#151, sum(profit#43)#152] Results [5]: [channel#134, id#135, sum(sales#41)#150 AS sales#153, sum(returns#42)#151 AS returns#154, sum(profit#43)#152 AS profit#155] -(82) TakeOrderedAndProject +(81) TakeOrderedAndProject Input [5]: [channel#134, id#135, sales#153, returns#154, profit#155] Arguments: 100, [channel#134 ASC NULLS FIRST, id#135 ASC NULLS FIRST], [channel#134, id#135, sales#153, returns#154, profit#155] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5 -ReusedExchange (83) +ReusedExchange (82) -(83) ReusedExchange [Reuses operator id: 20] +(82) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#49 IN dynamicpruning#50 -ReusedExchange (84) +ReusedExchange (83) -(84) ReusedExchange [Reuses operator id: 20] +(83) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt index d3097eff9f8bf..6239f80c2d62d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt @@ -114,11 +114,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Exchange [wr_item_sk,wr_order_number] #8 WholeStageCodegen (14) - Filter [wr_item_sk,wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter WholeStageCodegen (17) Sort [ws_item_sk,ws_order_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index 81cded23e42db..b3eb1f9592ff3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (79) -+- * HashAggregate (78) - +- Exchange (77) - +- * HashAggregate (76) - +- * Expand (75) - +- Union (74) +TakeOrderedAndProject (78) ++- * HashAggregate (77) + +- Exchange (76) + +- * HashAggregate (75) + +- * Expand (74) + +- Union (73) :- * HashAggregate (25) : +- Exchange (24) : +- * HashAggregate (23) @@ -51,41 +51,39 @@ TakeOrderedAndProject (79) : +- * Filter (40) : +- * ColumnarToRow (39) : +- Scan parquet default.catalog_page (38) - +- * HashAggregate (73) - +- Exchange (72) - +- * HashAggregate (71) - +- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- Union (61) + +- * HashAggregate (72) + +- Exchange (71) + +- * HashAggregate (70) + +- * Project (69) + +- * BroadcastHashJoin Inner BuildRight (68) + :- * Project (63) + : +- * BroadcastHashJoin Inner BuildRight (62) + : :- Union (60) : : :- * Project (50) : : : +- * Filter (49) : : : +- * ColumnarToRow (48) : : : +- Scan parquet default.web_sales (47) - : : +- * Project (60) - : : +- * BroadcastHashJoin Inner BuildLeft (59) - : : :- BroadcastExchange (54) - : : : +- * Filter (53) - : : : +- * ColumnarToRow (52) - : : : +- Scan parquet default.web_returns (51) - : : +- * Project (58) - : : +- * Filter (57) - : : +- * ColumnarToRow (56) - : : +- Scan parquet default.web_sales (55) - : +- ReusedExchange (62) - +- BroadcastExchange (68) - +- * Filter (67) - +- * ColumnarToRow (66) - +- Scan parquet default.web_site (65) + : : +- * Project (59) + : : +- * BroadcastHashJoin Inner BuildLeft (58) + : : :- BroadcastExchange (53) + : : : +- * ColumnarToRow (52) + : : : +- Scan parquet default.web_returns (51) + : : +- * Project (57) + : : +- * Filter (56) + : : +- * ColumnarToRow (55) + : : +- Scan parquet default.web_sales (54) + : +- ReusedExchange (61) + +- BroadcastExchange (67) + +- * Filter (66) + +- * ColumnarToRow (65) + +- Scan parquet default.web_site (64) (1) Scan parquet default.store_sales Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), isnotnull(cast(ss_sold_date_sk#4 as bigint)), dynamicpruningexpression(cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] +PartitionFilters: [isnotnull(cast(ss_sold_date_sk#4 as bigint)), dynamicpruningexpression(cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -93,7 +91,7 @@ Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_s (3) Filter [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_store_sk#1) AND isnotnull(cast(ss_store_sk#1 as bigint))) +Condition : isnotnull(cast(ss_store_sk#1 as bigint)) (4) Project [codegen id : 1] Output [6]: [cast(ss_store_sk#1 as bigint) AS store_sk#6, cast(ss_sold_date_sk#4 as bigint) AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] @@ -297,7 +295,7 @@ Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#8 Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct @@ -317,150 +315,145 @@ Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#1 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -(53) Filter [codegen id : 14] +(53) BroadcastExchange Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Condition : (isnotnull(wr_item_sk#98) AND isnotnull(wr_order_number#99)) +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] -(54) BroadcastExchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false], input[1, bigint, false]),false), [id=#103] - -(55) Scan parquet default.web_sales +(54) Scan parquet default.web_sales Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(56) ColumnarToRow +(55) ColumnarToRow Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(57) Filter +(56) Filter Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) -(58) Project +(57) Project Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(59) BroadcastHashJoin [codegen id : 15] +(58) BroadcastHashJoin [codegen id : 15] Left keys [2]: [wr_item_sk#98, wr_order_number#99] Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None -(60) Project [codegen id : 15] +(59) Project [codegen id : 15] Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -(61) Union +(60) Union -(62) ReusedExchange [Reuses operator id: 14] +(61) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] -(63) BroadcastHashJoin [codegen id : 18] +(62) BroadcastHashJoin [codegen id : 18] Left keys [1]: [date_sk#93] Right keys [1]: [cast(d_date_sk#22 as bigint)] Join condition: None -(64) Project [codegen id : 18] +(63) Project [codegen id : 18] Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] -(65) Scan parquet default.web_site +(64) Scan parquet default.web_site Output [2]: [web_site_sk#112, web_site_id#113] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 17] +(65) ColumnarToRow [codegen id : 17] Input [2]: [web_site_sk#112, web_site_id#113] -(67) Filter [codegen id : 17] +(66) Filter [codegen id : 17] Input [2]: [web_site_sk#112, web_site_id#113] Condition : isnotnull(web_site_sk#112) -(68) BroadcastExchange +(67) BroadcastExchange Input [2]: [web_site_sk#112, web_site_id#113] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] -(69) BroadcastHashJoin [codegen id : 18] +(68) BroadcastHashJoin [codegen id : 18] Left keys [1]: [wsr_web_site_sk#92] Right keys [1]: [web_site_sk#112] Join condition: None -(70) Project [codegen id : 18] +(69) Project [codegen id : 18] Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] -(71) HashAggregate [codegen id : 18] +(70) HashAggregate [codegen id : 18] Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] Keys [1]: [web_site_id#113] Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -(72) Exchange +(71) Exchange Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] -(73) HashAggregate [codegen id : 19] +(72) HashAggregate [codegen id : 19] Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] Keys [1]: [web_site_id#113] Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#128, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS RETURNS#129, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#130, web channel AS channel#131, concat(web_site, web_site_id#113) AS id#132] -(74) Union +(73) Union -(75) Expand [codegen id : 20] +(74) Expand [codegen id : 20] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] -(76) HashAggregate [codegen id : 20] +(75) HashAggregate [codegen id : 20] Input [6]: [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] Keys [3]: [channel#133, id#134, spark_grouping_id#135] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] Aggregate Attributes [6]: [sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Results [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -(77) Exchange +(76) Exchange Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] Arguments: hashpartitioning(channel#133, id#134, spark_grouping_id#135, 5), ENSURE_REQUIREMENTS, [id=#148] -(78) HashAggregate [codegen id : 21] +(77) HashAggregate [codegen id : 21] Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] Keys [3]: [channel#133, id#134, spark_grouping_id#135] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] Aggregate Attributes [3]: [sum(sales#41)#149, sum(returns#42)#150, sum(profit#43)#151] Results [5]: [channel#133, id#134, sum(sales#41)#149 AS sales#152, sum(returns#42)#150 AS returns#153, sum(profit#43)#151 AS profit#154] -(79) TakeOrderedAndProject +(78) TakeOrderedAndProject Input [5]: [channel#133, id#134, sales#152, returns#153, profit#154] Arguments: 100, [channel#133 ASC NULLS FIRST, id#134 ASC NULLS FIRST], [channel#133, id#134, sales#152, returns#153, profit#154] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5 -ReusedExchange (80) +ReusedExchange (79) -(80) ReusedExchange [Reuses operator id: 14] +(79) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#49 IN dynamicpruning#50 -ReusedExchange (81) +ReusedExchange (80) -(81) ReusedExchange [Reuses operator id: 14] +(80) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index 3fd10cf9cb948..50dde50e729d1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -111,11 +111,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #8 WholeStageCodegen (14) - Filter [wr_item_sk,wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 Project [ws_item_sk,ws_web_site_sk,ws_order_number] Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 322a882f92c48..8e3bf5bf57bcf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_item_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item Output [2]: [i_item_id#14, i_color#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [slate ,blanched ,burnished ]), IsNotNull(i_item_id)] +PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_item_id#14, i_color#15] (23) Filter [codegen id : 3] Input [2]: [i_item_id#14, i_color#15] -Condition : (i_color#15 IN (slate ,blanched ,burnished ) AND isnotnull(i_item_id#14)) +Condition : i_color#15 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] Output [1]: [i_item_id#14 AS i_item_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt index bfec5dcfdf5d0..2d21010c26b8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk,i_item_id] + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_color,i_item_id] + Filter [i_color] ColumnarToRow InputAdapter Scan parquet default.item [i_item_id,i_color] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 322a882f92c48..8e3bf5bf57bcf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_item_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item Output [2]: [i_item_id#14, i_color#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [slate ,blanched ,burnished ]), IsNotNull(i_item_id)] +PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_item_id#14, i_color#15] (23) Filter [codegen id : 3] Input [2]: [i_item_id#14, i_color#15] -Condition : (i_color#15 IN (slate ,blanched ,burnished ) AND isnotnull(i_item_id#14)) +Condition : i_color#15 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] Output [1]: [i_item_id#14 AS i_item_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index bfec5dcfdf5d0..2d21010c26b8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk,i_item_id] + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_color,i_item_id] + Filter [i_color] ColumnarToRow InputAdapter Scan parquet default.item [i_item_id,i_color] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index 5c08ac23ebb55..f6dd5a59de4d4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -84,7 +84,7 @@ TakeOrderedAndProject (79) Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct @@ -96,406 +96,385 @@ Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) (4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] +Output [2]: [d_date_sk#4, d_date#5] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#5, d_date#6] +Input [2]: [d_date_sk#4, d_date#5] (6) Filter [codegen id : 2] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#6)) +Input [2]: [d_date_sk#4, d_date#5] +Condition : isnotnull(d_date_sk#4) (7) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] +Input [2]: [d_date#5, d_week_seq#6] (9) Filter [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] -Condition : ((isnotnull(d_week_seq#7) AND (d_week_seq#7 = Subquery scalar-subquery#8, [id=#9])) AND isnotnull(d_date#6)) +Input [2]: [d_date#5, d_week_seq#6] +Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = Subquery scalar-subquery#7, [id=#8])) (10) Project [codegen id : 1] -Output [1]: [d_date#6 AS d_date#6#10] -Input [2]: [d_date#6, d_week_seq#7] +Output [1]: [d_date#5 AS d_date#5#9] +Input [2]: [d_date#5, d_week_seq#6] (11) BroadcastExchange -Input [1]: [d_date#6#10] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#11] +Input [1]: [d_date#5#9] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] (12) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_date#6] -Right keys [1]: [d_date#6#10] +Left keys [1]: [d_date#5] +Right keys [1]: [d_date#5#9] Join condition: None (13) Project [codegen id : 2] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +Output [1]: [d_date_sk#4] +Input [2]: [d_date_sk#4, d_date#5] (14) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +Input [1]: [d_date_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (15) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Right keys [1]: [d_date_sk#4] Join condition: None (16) Project [codegen id : 4] Output [2]: [ss_item_sk#1, ss_ext_sales_price#2] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#4] (17) Scan parquet default.item -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#12, i_item_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#13, i_item_id#14] +Input [2]: [i_item_sk#12, i_item_id#13] (19) Filter [codegen id : 3] -Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +Input [2]: [i_item_sk#12, i_item_id#13] +Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) (20) BroadcastExchange -Input [2]: [i_item_sk#13, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] +Input [2]: [i_item_sk#12, i_item_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] (21) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#12] Join condition: None (22) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#2, i_item_id#14] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, i_item_sk#13, i_item_id#14] +Output [2]: [ss_ext_sales_price#2, i_item_id#13] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, i_item_sk#12, i_item_id#13] (23) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#2, i_item_id#14] -Keys [1]: [i_item_id#14] +Input [2]: [ss_ext_sales_price#2, i_item_id#13] +Keys [1]: [i_item_id#13] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#14, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#13, sum#16] (24) Exchange -Input [2]: [i_item_id#14, sum#17] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [2]: [i_item_id#13, sum#16] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] (25) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#14, sum#17] -Keys [1]: [i_item_id#14] +Input [2]: [i_item_id#13, sum#16] +Keys [1]: [i_item_id#13] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#19] -Results [2]: [i_item_id#14 AS item_id#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS ss_item_rev#21] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#18] +Results [2]: [i_item_id#13 AS item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#18,17,2) AS ss_item_rev#20] (26) Filter [codegen id : 15] -Input [2]: [item_id#20, ss_item_rev#21] -Condition : isnotnull(ss_item_rev#21) +Input [2]: [item_id#19, ss_item_rev#20] +Condition : isnotnull(ss_item_rev#20) (27) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +Output [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#24), dynamicpruningexpression(cs_sold_date_sk#24 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] (29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] -Condition : isnotnull(cs_item_sk#22) +Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#21) (30) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] +Output [2]: [d_date_sk#4, d_date#5] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#5, d_date#6] +Input [2]: [d_date_sk#4, d_date#5] (32) Filter [codegen id : 6] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#6)) +Input [2]: [d_date_sk#4, d_date#5] +Condition : isnotnull(d_date_sk#4) (33) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (34) ColumnarToRow [codegen id : 5] -Input [2]: [d_date#6, d_week_seq#7] +Input [2]: [d_date#5, d_week_seq#6] (35) Filter [codegen id : 5] -Input [2]: [d_date#6, d_week_seq#7] -Condition : ((isnotnull(d_week_seq#7) AND (d_week_seq#7 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) AND isnotnull(d_date#6)) +Input [2]: [d_date#5, d_week_seq#6] +Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) (36) Project [codegen id : 5] -Output [1]: [d_date#6 AS d_date#6#26] -Input [2]: [d_date#6, d_week_seq#7] +Output [1]: [d_date#5 AS d_date#5#24] +Input [2]: [d_date#5, d_week_seq#6] (37) BroadcastExchange -Input [1]: [d_date#6#26] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#27] +Input [1]: [d_date#5#24] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] (38) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_date#6] -Right keys [1]: [d_date#6#26] +Left keys [1]: [d_date#5] +Right keys [1]: [d_date#5#24] Join condition: None (39) Project [codegen id : 6] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +Output [1]: [d_date_sk#4] +Input [2]: [d_date_sk#4, d_date#5] (40) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [d_date_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#24] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#4] Join condition: None (42) Project [codegen id : 8] -Output [2]: [cs_item_sk#22, cs_ext_sales_price#23] -Input [4]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24, d_date_sk#5] +Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#4] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#12, i_item_id#13] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#22] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [cs_item_sk#21] +Right keys [1]: [i_item_sk#12] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#23, i_item_id#14] -Input [4]: [cs_item_sk#22, cs_ext_sales_price#23, i_item_sk#13, i_item_id#14] +Output [2]: [cs_ext_sales_price#22, i_item_id#13] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#12, i_item_id#13] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#23, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum#29] -Results [2]: [i_item_id#14, sum#30] +Input [2]: [cs_ext_sales_price#22, i_item_id#13] +Keys [1]: [i_item_id#13] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] +Aggregate Attributes [1]: [sum#27] +Results [2]: [i_item_id#13, sum#28] (47) Exchange -Input [2]: [i_item_id#14, sum#30] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [i_item_id#13, sum#28] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#29] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#14, sum#30] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#23))#32] -Results [2]: [i_item_id#14 AS item_id#33, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#23))#32,17,2) AS cs_item_rev#34] +Input [2]: [i_item_id#13, sum#28] +Keys [1]: [i_item_id#13] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] +Results [2]: [i_item_id#13 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] (49) Filter [codegen id : 9] -Input [2]: [item_id#33, cs_item_rev#34] -Condition : isnotnull(cs_item_rev#34) +Input [2]: [item_id#31, cs_item_rev#32] +Condition : isnotnull(cs_item_rev#32) (50) BroadcastExchange -Input [2]: [item_id#33, cs_item_rev#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#35] +Input [2]: [item_id#31, cs_item_rev#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#33] -Join condition: ((((cast(ss_item_rev#21 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#34)), DecimalType(19,3), true)) AND (cast(ss_item_rev#21 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#34)), DecimalType(20,3), true))) AND (cast(cs_item_rev#34 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#21)), DecimalType(19,3), true))) AND (cast(cs_item_rev#34 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#21)), DecimalType(20,3), true))) +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#31] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#20, ss_item_rev#21, cs_item_rev#34] -Input [4]: [item_id#20, ss_item_rev#21, item_id#33, cs_item_rev#34] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] +Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] +Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_item_sk#36) +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#34) (56) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] +Output [2]: [d_date_sk#4, d_date#5] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 11] -Input [2]: [d_date_sk#5, d_date#6] +Input [2]: [d_date_sk#4, d_date#5] (58) Filter [codegen id : 11] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#6)) +Input [2]: [d_date_sk#4, d_date#5] +Condition : isnotnull(d_date_sk#4) (59) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (60) ColumnarToRow [codegen id : 10] -Input [2]: [d_date#6, d_week_seq#7] +Input [2]: [d_date#5, d_week_seq#6] (61) Filter [codegen id : 10] -Input [2]: [d_date#6, d_week_seq#7] -Condition : ((isnotnull(d_week_seq#7) AND (d_week_seq#7 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) AND isnotnull(d_date#6)) +Input [2]: [d_date#5, d_week_seq#6] +Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) (62) Project [codegen id : 10] -Output [1]: [d_date#6 AS d_date#6#40] -Input [2]: [d_date#6, d_week_seq#7] +Output [1]: [d_date#5 AS d_date#5#37] +Input [2]: [d_date#5, d_week_seq#6] (63) BroadcastExchange -Input [1]: [d_date#6#40] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#41] +Input [1]: [d_date#5#37] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] (64) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_date#6] -Right keys [1]: [d_date#6#40] +Left keys [1]: [d_date#5] +Right keys [1]: [d_date#5#37] Join condition: None (65) Project [codegen id : 11] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +Output [1]: [d_date_sk#4] +Input [2]: [d_date_sk#4, d_date#5] (66) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [d_date_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] (67) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#4] Join condition: None (68) Project [codegen id : 13] -Output [2]: [ws_item_sk#36, ws_ext_sales_price#37] -Input [4]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38, d_date_sk#5] +Output [2]: [ws_item_sk#34, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#4] (69) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#12, i_item_id#13] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#36] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#12] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#37, i_item_id#14] -Input [4]: [ws_item_sk#36, ws_ext_sales_price#37, i_item_sk#13, i_item_id#14] +Output [2]: [ws_ext_sales_price#35, i_item_id#13] +Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, i_item_sk#12, i_item_id#13] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#37, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#37))] -Aggregate Attributes [1]: [sum#43] -Results [2]: [i_item_id#14, sum#44] +Input [2]: [ws_ext_sales_price#35, i_item_id#13] +Keys [1]: [i_item_id#13] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#40] +Results [2]: [i_item_id#13, sum#41] (73) Exchange -Input [2]: [i_item_id#14, sum#44] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [2]: [i_item_id#13, sum#41] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#42] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#14, sum#44] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#37))#46] -Results [2]: [i_item_id#14 AS item_id#47, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#37))#46,17,2) AS ws_item_rev#48] +Input [2]: [i_item_id#13, sum#41] +Keys [1]: [i_item_id#13] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] +Results [2]: [i_item_id#13 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] (75) Filter [codegen id : 14] -Input [2]: [item_id#47, ws_item_rev#48] -Condition : isnotnull(ws_item_rev#48) +Input [2]: [item_id#44, ws_item_rev#45] +Condition : isnotnull(ws_item_rev#45) (76) BroadcastExchange -Input [2]: [item_id#47, ws_item_rev#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#49] +Input [2]: [item_id#44, ws_item_rev#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] (77) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#47] -Join condition: ((((((((cast(ss_item_rev#21 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#48)), DecimalType(19,3), true)) AND (cast(ss_item_rev#21 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#48)), DecimalType(20,3), true))) AND (cast(cs_item_rev#34 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#48)), DecimalType(19,3), true))) AND (cast(cs_item_rev#34 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#48)), DecimalType(20,3), true))) AND (cast(ws_item_rev#48 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#21)), DecimalType(19,3), true))) AND (cast(ws_item_rev#48 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#21)), DecimalType(20,3), true))) AND (cast(ws_item_rev#48 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#34)), DecimalType(19,3), true))) AND (cast(ws_item_rev#48 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#34)), DecimalType(20,3), true))) +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#44] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#20, ss_item_rev#21, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#50, cs_item_rev#34, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#34 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#51, ws_item_rev#48, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#48 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#52, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#53] -Input [5]: [item_id#20, ss_item_rev#21, cs_item_rev#34, item_id#47, ws_item_rev#48] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] (79) TakeOrderedAndProject -Input [8]: [item_id#20, ss_item_rev#21, ss_dev#50, cs_item_rev#34, cs_dev#51, ws_item_rev#48, ws_dev#52, average#53] -Arguments: 100, [item_id#20 ASC NULLS FIRST, ss_item_rev#21 ASC NULLS FIRST], [item_id#20, ss_item_rev#21, ss_dev#50, cs_item_rev#34, cs_dev#51, ws_item_rev#48, ws_dev#52, average#53] +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] ===== Subqueries ===== -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -ReusedExchange (80) +Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#7, [id=#8] +* Project (83) ++- * Filter (82) + +- * ColumnarToRow (81) + +- Scan parquet default.date_dim (80) -(80) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#5] - -Subquery:2 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* Project (84) -+- * Filter (83) - +- * ColumnarToRow (82) - +- Scan parquet default.date_dim (81) - - -(81) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +(80) Scan parquet default.date_dim +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(82) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] - -(83) Filter [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] -Condition : (isnotnull(d_date#6) AND (d_date#6 = 2000-01-03)) - -(84) Project [codegen id : 1] -Output [1]: [d_week_seq#7] -Input [2]: [d_date#6, d_week_seq#7] - -Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#24 IN dynamicpruning#25 -ReusedExchange (85) - - -(85) ReusedExchange [Reuses operator id: unknown] -Output [1]: [d_date_sk#5] - -Subquery:4 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] +(81) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#5, d_week_seq#6] -Subquery:5 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#39 -ReusedExchange (86) +(82) Filter [codegen id : 1] +Input [2]: [d_date#5, d_week_seq#6] +Condition : (isnotnull(d_date#5) AND (d_date#5 = 2000-01-03)) +(83) Project [codegen id : 1] +Output [1]: [d_week_seq#6] +Input [2]: [d_date#5, d_week_seq#6] -(86) ReusedExchange [Reuses operator id: unknown] -Output [1]: [d_date_sk#5] +Subquery:2 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] -Subquery:6 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] +Subquery:3 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt index 67f864915caca..5642c7dcbf91c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt @@ -18,14 +18,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #2 WholeStageCodegen (2) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] @@ -33,8 +31,8 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev BroadcastExchange #3 WholeStageCodegen (1) Project [d_date] - Filter [d_week_seq,d_date] - Subquery #2 + Filter [d_week_seq] + Subquery #1 WholeStageCodegen (1) Project [d_week_seq] Filter [d_date] @@ -68,35 +66,33 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - ReusedExchange [d_date_sk] #7 InputAdapter - BroadcastExchange #8 + BroadcastExchange #7 WholeStageCodegen (6) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #9 + BroadcastExchange #8 WholeStageCodegen (5) Project [d_date] - Filter [d_week_seq,d_date] - ReusedSubquery [d_week_seq] #2 + Filter [d_week_seq] + ReusedSubquery [d_week_seq] #1 ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter ReusedExchange [i_item_sk,i_item_id] #4 InputAdapter - BroadcastExchange #10 + BroadcastExchange #9 WholeStageCodegen (14) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] InputAdapter - Exchange [i_item_id] #11 + Exchange [i_item_id] #10 WholeStageCodegen (13) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -107,23 +103,21 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - ReusedExchange [d_date_sk] #12 InputAdapter - BroadcastExchange #13 + BroadcastExchange #11 WholeStageCodegen (11) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #14 + BroadcastExchange #12 WholeStageCodegen (10) Project [d_date] - Filter [d_week_seq,d_date] - ReusedSubquery [d_week_seq] #2 + Filter [d_week_seq] + ReusedSubquery [d_week_seq] #1 ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index 499559c701c71..ab4c4ad4ae65b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -84,7 +84,7 @@ TakeOrderedAndProject (79) Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct @@ -96,406 +96,385 @@ Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) (4) Scan parquet default.item -Output [2]: [i_item_sk#5, i_item_id#6] +Output [2]: [i_item_sk#4, i_item_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#6] +Input [2]: [i_item_sk#4, i_item_id#5] (6) Filter [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) +Input [2]: [i_item_sk#4, i_item_id#5] +Condition : (isnotnull(i_item_sk#4) AND isnotnull(i_item_id#5)) (7) BroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] +Input [2]: [i_item_sk#4, i_item_id#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#6] (8) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] +Right keys [1]: [i_item_sk#4] Join condition: None (9) Project [codegen id : 4] -Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6] -Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6] +Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#5] +Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#4, i_item_id#5] (10) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (11) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#7, d_date#8] (12) Filter [codegen id : 3] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (isnotnull(d_date_sk#8) AND isnotnull(d_date#9)) +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) (13) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (14) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#9, d_week_seq#10] +Input [2]: [d_date#8, d_week_seq#9] (15) Filter [codegen id : 2] -Input [2]: [d_date#9, d_week_seq#10] -Condition : ((isnotnull(d_week_seq#10) AND (d_week_seq#10 = Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date#9)) +Input [2]: [d_date#8, d_week_seq#9] +Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = Subquery scalar-subquery#10, [id=#11])) (16) Project [codegen id : 2] -Output [1]: [d_date#9 AS d_date#9#13] -Input [2]: [d_date#9, d_week_seq#10] +Output [1]: [d_date#8 AS d_date#8#12] +Input [2]: [d_date#8, d_week_seq#9] (17) BroadcastExchange -Input [1]: [d_date#9#13] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#14] +Input [1]: [d_date#8#12] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] (18) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date#9] -Right keys [1]: [d_date#9#13] +Left keys [1]: [d_date#8] +Right keys [1]: [d_date#8#12] Join condition: None (19) Project [codegen id : 3] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#7] +Input [2]: [d_date_sk#7, d_date#8] (20) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (21) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#7] Join condition: None (22) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#2, i_item_id#6] -Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, d_date_sk#8] +Output [2]: [ss_ext_sales_price#2, i_item_id#5] +Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#5, d_date_sk#7] (23) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#2, i_item_id#6] -Keys [1]: [i_item_id#6] +Input [2]: [ss_ext_sales_price#2, i_item_id#5] +Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#6, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#5, sum#16] (24) Exchange -Input [2]: [i_item_id#6, sum#17] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [2]: [i_item_id#5, sum#16] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#17] (25) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#6, sum#17] -Keys [1]: [i_item_id#6] +Input [2]: [i_item_id#5, sum#16] +Keys [1]: [i_item_id#5] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#19] -Results [2]: [i_item_id#6 AS item_id#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS ss_item_rev#21] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#18] +Results [2]: [i_item_id#5 AS item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#18,17,2) AS ss_item_rev#20] (26) Filter [codegen id : 15] -Input [2]: [item_id#20, ss_item_rev#21] -Condition : isnotnull(ss_item_rev#21) +Input [2]: [item_id#19, ss_item_rev#20] +Condition : isnotnull(ss_item_rev#20) (27) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +Output [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#24), dynamicpruningexpression(cs_sold_date_sk#24 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] (29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] -Condition : isnotnull(cs_item_sk#22) +Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#21) (30) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_item_id#6] +Output [2]: [i_item_sk#4, i_item_id#5] (31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#22] -Right keys [1]: [i_item_sk#5] +Left keys [1]: [cs_item_sk#21] +Right keys [1]: [i_item_sk#4] Join condition: None (32) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#23, cs_sold_date_sk#24, i_item_id#6] -Input [5]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24, i_item_sk#5, i_item_id#6] +Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5] +Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#4, i_item_id#5] (33) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#7, d_date#8] (35) Filter [codegen id : 7] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (isnotnull(d_date_sk#8) AND isnotnull(d_date#9)) +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) (36) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [2]: [d_date#9, d_week_seq#10] +Input [2]: [d_date#8, d_week_seq#9] (38) Filter [codegen id : 6] -Input [2]: [d_date#9, d_week_seq#10] -Condition : ((isnotnull(d_week_seq#10) AND (d_week_seq#10 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date#9)) +Input [2]: [d_date#8, d_week_seq#9] +Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) (39) Project [codegen id : 6] -Output [1]: [d_date#9 AS d_date#9#26] -Input [2]: [d_date#9, d_week_seq#10] +Output [1]: [d_date#8 AS d_date#8#24] +Input [2]: [d_date#8, d_week_seq#9] (40) BroadcastExchange -Input [1]: [d_date#9#26] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#27] +Input [1]: [d_date#8#24] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [d_date#9] -Right keys [1]: [d_date#9#26] +Left keys [1]: [d_date#8] +Right keys [1]: [d_date#8#24] Join condition: None (42) Project [codegen id : 7] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#7] +Input [2]: [d_date_sk#7, d_date#8] (43) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#24] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#7] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#23, i_item_id#6] -Input [4]: [cs_ext_sales_price#23, cs_sold_date_sk#24, i_item_id#6, d_date_sk#8] +Output [2]: [cs_ext_sales_price#22, i_item_id#5] +Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5, d_date_sk#7] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#23, i_item_id#6] -Keys [1]: [i_item_id#6] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum#29] -Results [2]: [i_item_id#6, sum#30] +Input [2]: [cs_ext_sales_price#22, i_item_id#5] +Keys [1]: [i_item_id#5] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] +Aggregate Attributes [1]: [sum#27] +Results [2]: [i_item_id#5, sum#28] (47) Exchange -Input [2]: [i_item_id#6, sum#30] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [i_item_id#5, sum#28] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#29] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#6, sum#30] -Keys [1]: [i_item_id#6] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#23))#32] -Results [2]: [i_item_id#6 AS item_id#33, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#23))#32,17,2) AS cs_item_rev#34] +Input [2]: [i_item_id#5, sum#28] +Keys [1]: [i_item_id#5] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] +Results [2]: [i_item_id#5 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] (49) Filter [codegen id : 9] -Input [2]: [item_id#33, cs_item_rev#34] -Condition : isnotnull(cs_item_rev#34) +Input [2]: [item_id#31, cs_item_rev#32] +Condition : isnotnull(cs_item_rev#32) (50) BroadcastExchange -Input [2]: [item_id#33, cs_item_rev#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#35] +Input [2]: [item_id#31, cs_item_rev#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#33] -Join condition: ((((cast(ss_item_rev#21 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#34)), DecimalType(19,3), true)) AND (cast(ss_item_rev#21 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#34)), DecimalType(20,3), true))) AND (cast(cs_item_rev#34 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#21)), DecimalType(19,3), true))) AND (cast(cs_item_rev#34 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#21)), DecimalType(20,3), true))) +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#31] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#20, ss_item_rev#21, cs_item_rev#34] -Input [4]: [item_id#20, ss_item_rev#21, item_id#33, cs_item_rev#34] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] +Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] +Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_item_sk#36) +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#34) (56) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_item_id#6] +Output [2]: [i_item_sk#4, i_item_id#5] (57) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#36] -Right keys [1]: [i_item_sk#5] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#4] Join condition: None (58) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#37, ws_sold_date_sk#38, i_item_id#6] -Input [5]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38, i_item_sk#5, i_item_id#6] +Output [3]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5] +Input [5]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_sk#4, i_item_id#5] (59) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#7, d_date#8] (61) Filter [codegen id : 12] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (isnotnull(d_date_sk#8) AND isnotnull(d_date#9)) +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) (62) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (63) ColumnarToRow [codegen id : 11] -Input [2]: [d_date#9, d_week_seq#10] +Input [2]: [d_date#8, d_week_seq#9] (64) Filter [codegen id : 11] -Input [2]: [d_date#9, d_week_seq#10] -Condition : ((isnotnull(d_week_seq#10) AND (d_week_seq#10 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date#9)) +Input [2]: [d_date#8, d_week_seq#9] +Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) (65) Project [codegen id : 11] -Output [1]: [d_date#9 AS d_date#9#40] -Input [2]: [d_date#9, d_week_seq#10] +Output [1]: [d_date#8 AS d_date#8#37] +Input [2]: [d_date#8, d_week_seq#9] (66) BroadcastExchange -Input [1]: [d_date#9#40] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#41] +Input [1]: [d_date#8#37] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] (67) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_date#9] -Right keys [1]: [d_date#9#40] +Left keys [1]: [d_date#8] +Right keys [1]: [d_date#8#37] Join condition: None (68) Project [codegen id : 12] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#7] +Input [2]: [d_date_sk#7, d_date#8] (69) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#7] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#37, i_item_id#6] -Input [4]: [ws_ext_sales_price#37, ws_sold_date_sk#38, i_item_id#6, d_date_sk#8] +Output [2]: [ws_ext_sales_price#35, i_item_id#5] +Input [4]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5, d_date_sk#7] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#37, i_item_id#6] -Keys [1]: [i_item_id#6] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#37))] -Aggregate Attributes [1]: [sum#43] -Results [2]: [i_item_id#6, sum#44] +Input [2]: [ws_ext_sales_price#35, i_item_id#5] +Keys [1]: [i_item_id#5] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#40] +Results [2]: [i_item_id#5, sum#41] (73) Exchange -Input [2]: [i_item_id#6, sum#44] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [2]: [i_item_id#5, sum#41] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#42] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#6, sum#44] -Keys [1]: [i_item_id#6] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#37))#46] -Results [2]: [i_item_id#6 AS item_id#47, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#37))#46,17,2) AS ws_item_rev#48] +Input [2]: [i_item_id#5, sum#41] +Keys [1]: [i_item_id#5] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] +Results [2]: [i_item_id#5 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] (75) Filter [codegen id : 14] -Input [2]: [item_id#47, ws_item_rev#48] -Condition : isnotnull(ws_item_rev#48) +Input [2]: [item_id#44, ws_item_rev#45] +Condition : isnotnull(ws_item_rev#45) (76) BroadcastExchange -Input [2]: [item_id#47, ws_item_rev#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#49] +Input [2]: [item_id#44, ws_item_rev#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] (77) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#47] -Join condition: ((((((((cast(ss_item_rev#21 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#48)), DecimalType(19,3), true)) AND (cast(ss_item_rev#21 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#48)), DecimalType(20,3), true))) AND (cast(cs_item_rev#34 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#48)), DecimalType(19,3), true))) AND (cast(cs_item_rev#34 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#48)), DecimalType(20,3), true))) AND (cast(ws_item_rev#48 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#21)), DecimalType(19,3), true))) AND (cast(ws_item_rev#48 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#21)), DecimalType(20,3), true))) AND (cast(ws_item_rev#48 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#34)), DecimalType(19,3), true))) AND (cast(ws_item_rev#48 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#34)), DecimalType(20,3), true))) +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#44] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#20, ss_item_rev#21, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#50, cs_item_rev#34, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#34 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#51, ws_item_rev#48, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#48 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#52, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#53] -Input [5]: [item_id#20, ss_item_rev#21, cs_item_rev#34, item_id#47, ws_item_rev#48] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] (79) TakeOrderedAndProject -Input [8]: [item_id#20, ss_item_rev#21, ss_dev#50, cs_item_rev#34, cs_dev#51, ws_item_rev#48, ws_dev#52, average#53] -Arguments: 100, [item_id#20 ASC NULLS FIRST, ss_item_rev#21 ASC NULLS FIRST], [item_id#20, ss_item_rev#21, ss_dev#50, cs_item_rev#34, cs_dev#51, ws_item_rev#48, ws_dev#52, average#53] +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] ===== Subqueries ===== -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -ReusedExchange (80) +Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#10, [id=#11] +* Project (83) ++- * Filter (82) + +- * ColumnarToRow (81) + +- Scan parquet default.date_dim (80) -(80) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#8] - -Subquery:2 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (84) -+- * Filter (83) - +- * ColumnarToRow (82) - +- Scan parquet default.date_dim (81) - - -(81) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +(80) Scan parquet default.date_dim +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(82) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#9, d_week_seq#10] - -(83) Filter [codegen id : 1] -Input [2]: [d_date#9, d_week_seq#10] -Condition : (isnotnull(d_date#9) AND (d_date#9 = 2000-01-03)) - -(84) Project [codegen id : 1] -Output [1]: [d_week_seq#10] -Input [2]: [d_date#9, d_week_seq#10] - -Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#24 IN dynamicpruning#25 -ReusedExchange (85) - - -(85) ReusedExchange [Reuses operator id: unknown] -Output [1]: [d_date_sk#8] - -Subquery:4 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] +(81) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#8, d_week_seq#9] -Subquery:5 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#39 -ReusedExchange (86) +(82) Filter [codegen id : 1] +Input [2]: [d_date#8, d_week_seq#9] +Condition : (isnotnull(d_date#8) AND (d_date#8 = 2000-01-03)) +(83) Project [codegen id : 1] +Output [1]: [d_week_seq#9] +Input [2]: [d_date#8, d_week_seq#9] -(86) ReusedExchange [Reuses operator id: unknown] -Output [1]: [d_date_sk#8] +Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] -Subquery:6 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] +Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index ea72681c2f5f1..b2e184c7e279d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -18,21 +18,19 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #3 + BroadcastExchange #2 WholeStageCodegen (1) Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #2 + BroadcastExchange #3 WholeStageCodegen (3) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] @@ -40,8 +38,8 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev BroadcastExchange #4 WholeStageCodegen (2) Project [d_date] - Filter [d_week_seq,d_date] - Subquery #2 + Filter [d_week_seq] + Subquery #1 WholeStageCodegen (1) Project [d_week_seq] Filter [d_date] @@ -68,35 +66,33 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - ReusedExchange [d_date_sk] #7 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #3 + ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - BroadcastExchange #8 + BroadcastExchange #7 WholeStageCodegen (7) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #9 + BroadcastExchange #8 WholeStageCodegen (6) Project [d_date] - Filter [d_week_seq,d_date] - ReusedSubquery [d_week_seq] #2 + Filter [d_week_seq] + ReusedSubquery [d_week_seq] #1 ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter - BroadcastExchange #10 + BroadcastExchange #9 WholeStageCodegen (14) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] InputAdapter - Exchange [i_item_id] #11 + Exchange [i_item_id] #10 WholeStageCodegen (13) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -107,25 +103,23 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - ReusedExchange [d_date_sk] #12 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #3 + ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - BroadcastExchange #13 + BroadcastExchange #11 WholeStageCodegen (12) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #14 + BroadcastExchange #12 WholeStageCodegen (11) Project [d_date] - Filter [d_week_seq,d_date] - ReusedSubquery [d_week_seq] #2 + Filter [d_week_seq] + ReusedSubquery [d_week_seq] #1 ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 0085627d7f485..10b8d0d9e7f05 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_item_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item Output [2]: [i_item_id#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music ), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_item_id#14, i_category#15] (23) Filter [codegen id : 3] Input [2]: [i_item_id#14, i_category#15] -Condition : ((isnotnull(i_category#15) AND (i_category#15 = Music )) AND isnotnull(i_item_id#14)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) (24) Project [codegen id : 3] Output [1]: [i_item_id#14 AS i_item_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt index 45041ad094f7b..46775417cbb70 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk,i_item_id] + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_category,i_item_id] + Filter [i_category] ColumnarToRow InputAdapter Scan parquet default.item [i_item_id,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 0085627d7f485..10b8d0d9e7f05 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_item_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item Output [2]: [i_item_id#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music ), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_item_id#14, i_category#15] (23) Filter [codegen id : 3] Input [2]: [i_item_id#14, i_category#15] -Condition : ((isnotnull(i_category#15) AND (i_category#15 = Music )) AND isnotnull(i_item_id#14)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) (24) Project [codegen id : 3] Output [1]: [i_item_id#14 AS i_item_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 45041ad094f7b..46775417cbb70 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk,i_item_id] + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_category,i_item_id] + Filter [i_category] ColumnarToRow InputAdapter Scan parquet default.item [i_item_id,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt index 58598de9e10c5..97d7a81baf2a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt @@ -1,65 +1,62 @@ == Physical Plan == -TakeOrderedAndProject (54) -+- * HashAggregate (53) - +- Exchange (52) - +- * HashAggregate (51) - +- * Project (50) - +- * BroadcastHashJoin Inner BuildLeft (49) - :- BroadcastExchange (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (37) - : : +- SortMergeJoin LeftAnti (36) - : : :- SortMergeJoin LeftAnti (27) - : : : :- SortMergeJoin LeftSemi (18) +TakeOrderedAndProject (51) ++- * HashAggregate (50) + +- Exchange (49) + +- * HashAggregate (48) + +- * Project (47) + +- * BroadcastHashJoin Inner BuildLeft (46) + :- BroadcastExchange (42) + : +- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (34) + : : +- SortMergeJoin LeftAnti (33) + : : :- SortMergeJoin LeftAnti (25) + : : : :- SortMergeJoin LeftSemi (17) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- * Sort (17) - : : : : +- Exchange (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Filter (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- Scan parquet default.store_sales (6) - : : : : +- BroadcastExchange (13) - : : : : +- * Project (12) - : : : : +- * Filter (11) - : : : : +- * ColumnarToRow (10) - : : : : +- Scan parquet default.date_dim (9) - : : : +- * Sort (26) - : : : +- Exchange (25) - : : : +- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (21) - : : : : +- * ColumnarToRow (20) - : : : : +- Scan parquet default.web_sales (19) - : : : +- ReusedExchange (22) - : : +- * Sort (35) - : : +- Exchange (34) - : : +- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet default.catalog_sales (28) - : : +- ReusedExchange (31) - : +- BroadcastExchange (42) - : +- * Project (41) - : +- * Filter (40) - : +- * ColumnarToRow (39) - : +- Scan parquet default.customer_address (38) - +- * Filter (48) - +- * ColumnarToRow (47) - +- Scan parquet default.customer_demographics (46) + : : : : +- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Project (14) + : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : :- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (12) + : : : : +- * Project (11) + : : : : +- * Filter (10) + : : : : +- * ColumnarToRow (9) + : : : : +- Scan parquet default.date_dim (8) + : : : +- * Sort (24) + : : : +- Exchange (23) + : : : +- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- Scan parquet default.web_sales (18) + : : : +- ReusedExchange (20) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * ColumnarToRow (27) + : : : +- Scan parquet default.catalog_sales (26) + : : +- ReusedExchange (28) + : +- BroadcastExchange (39) + : +- * Project (38) + : +- * Filter (37) + : +- * ColumnarToRow (36) + : +- Scan parquet default.customer_address (35) + +- * Filter (45) + +- * ColumnarToRow (44) + +- Scan parquet default.customer_demographics (43) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -67,7 +64,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -82,235 +79,220 @@ Output [2]: [ss_customer_sk#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -(8) Filter [codegen id : 4] -Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_customer_sk#5) - -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#8, d_year#9, d_moy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] Condition : (((((isnotnull(d_year#9) AND isnotnull(d_moy#10)) AND (d_year#9 = 2001)) AND (d_moy#10 >= 4)) AND (d_moy#10 <= 6)) AND isnotnull(d_date_sk#8)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [1]: [d_date_sk#8] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -(13) BroadcastExchange +(12) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#8] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [1]: [ss_customer_sk#5] Input [3]: [ss_customer_sk#5, ss_sold_date_sk#6, d_date_sk#8] -(16) Exchange +(15) Exchange Input [1]: [ss_customer_sk#5] Arguments: hashpartitioning(ss_customer_sk#5, 5), ENSURE_REQUIREMENTS, [id=#12] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [1]: [ss_customer_sk#5] Arguments: [ss_customer_sk#5 ASC NULLS FIRST], false, 0 -(18) SortMergeJoin +(17) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#5] Join condition: None -(19) Scan parquet default.web_sales +(18) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#14), dynamicpruningexpression(ws_sold_date_sk#14 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 7] -Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] - -(21) Filter [codegen id : 7] +(19) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] -Condition : isnotnull(ws_bill_customer_sk#13) -(22) ReusedExchange [Reuses operator id: 13] +(20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -(23) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] Right keys [1]: [d_date_sk#8] Join condition: None -(24) Project [codegen id : 7] +(22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13] Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] -(25) Exchange +(23) Exchange Input [1]: [ws_bill_customer_sk#13] Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#15] -(26) Sort [codegen id : 8] +(24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#13] Arguments: [ws_bill_customer_sk#13 ASC NULLS FIRST], false, 0 -(27) SortMergeJoin +(25) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#13] Join condition: None -(28) Scan parquet default.catalog_sales +(26) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 10] +(27) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -(30) Filter [codegen id : 10] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Condition : isnotnull(cs_ship_customer_sk#16) - -(31) ReusedExchange [Reuses operator id: 13] +(28) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -(32) BroadcastHashJoin [codegen id : 10] +(29) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#8] Join condition: None -(33) Project [codegen id : 10] +(30) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#16] Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] -(34) Exchange +(31) Exchange Input [1]: [cs_ship_customer_sk#16] Arguments: hashpartitioning(cs_ship_customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] -(35) Sort [codegen id : 11] +(32) Sort [codegen id : 11] Input [1]: [cs_ship_customer_sk#16] Arguments: [cs_ship_customer_sk#16 ASC NULLS FIRST], false, 0 -(36) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#16] Join condition: None -(37) Project [codegen id : 13] +(34) Project [codegen id : 13] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(38) Scan parquet default.customer_address +(35) Scan parquet default.customer_address Output [2]: [ca_address_sk#19, ca_state#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 12] +(36) ColumnarToRow [codegen id : 12] Input [2]: [ca_address_sk#19, ca_state#20] -(40) Filter [codegen id : 12] +(37) Filter [codegen id : 12] Input [2]: [ca_address_sk#19, ca_state#20] Condition : (ca_state#20 IN (KY,GA,NM) AND isnotnull(ca_address_sk#19)) -(41) Project [codegen id : 12] +(38) Project [codegen id : 12] Output [1]: [ca_address_sk#19] Input [2]: [ca_address_sk#19, ca_state#20] -(42) BroadcastExchange +(39) BroadcastExchange Input [1]: [ca_address_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(43) BroadcastHashJoin [codegen id : 13] +(40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#19] Join condition: None -(44) Project [codegen id : 13] +(41) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] -(45) BroadcastExchange +(42) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(46) Scan parquet default.customer_demographics +(43) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(47) ColumnarToRow +(44) ColumnarToRow Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -(48) Filter +(45) Filter Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Condition : isnotnull(cd_demo_sk#23) -(49) BroadcastHashJoin [codegen id : 14] +(46) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#23] Join condition: None -(50) Project [codegen id : 14] +(47) Project [codegen id : 14] Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -(51) HashAggregate [codegen id : 14] +(48) HashAggregate [codegen id : 14] Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#29] Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] -(52) Exchange +(49) Exchange Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, 5), ENSURE_REQUIREMENTS, [id=#31] -(53) HashAggregate [codegen id : 15] +(50) HashAggregate [codegen id : 15] Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#32] Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#32 AS cnt1#33, cd_purchase_estimate#27, count(1)#32 AS cnt2#34, cd_credit_rating#28, count(1)#32 AS cnt3#35] -(54) TakeOrderedAndProject +(51) TakeOrderedAndProject Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -ReusedExchange (55) +ReusedExchange (52) -(55) ReusedExchange [Reuses operator id: 13] +(52) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt index 9a333f419c399..dd7379392ec2f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -33,12 +33,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #5 WholeStageCodegen (3) @@ -54,11 +53,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (7) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (11) @@ -68,11 +66,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (10) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index 5e074df039648..b5307abaf688c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -1,60 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (32) - : : +- * BroadcastHashJoin LeftAnti BuildRight (31) - : : :- * BroadcastHashJoin LeftAnti BuildRight (23) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Project (29) + : : +- * BroadcastHashJoin LeftAnti BuildRight (28) + : : :- * BroadcastHashJoin LeftAnti BuildRight (21) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (14) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (14) - : : : : +- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (22) - : : : +- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (19) - : : +- BroadcastExchange (30) - : : +- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Filter (26) - : : : +- * ColumnarToRow (25) - : : : +- Scan parquet default.catalog_sales (24) - : : +- ReusedExchange (27) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- * Filter (35) - : +- * ColumnarToRow (34) - : +- Scan parquet default.customer_address (33) - +- BroadcastExchange (43) - +- * Filter (42) - +- * ColumnarToRow (41) - +- Scan parquet default.customer_demographics (40) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * ColumnarToRow (5) + : : : : : +- Scan parquet default.store_sales (4) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.date_dim (6) + : : : +- BroadcastExchange (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- BroadcastExchange (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (24) + : +- BroadcastExchange (34) + : +- * Project (33) + : +- * Filter (32) + : +- * ColumnarToRow (31) + : +- Scan parquet default.customer_address (30) + +- BroadcastExchange (40) + +- * Filter (39) + +- * ColumnarToRow (38) + +- Scan parquet default.customer_demographics (37) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -62,230 +59,215 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) Filter [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Condition : isnotnull(ss_customer_sk#4) - -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [1]: [d_date_sk#7] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -(11) BroadcastExchange +(10) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] -(12) BroadcastHashJoin [codegen id : 2] +(11) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join condition: None -(13) Project [codegen id : 2] +(12) Project [codegen id : 2] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -(14) BroadcastExchange +(13) BroadcastExchange Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(15) BroadcastHashJoin [codegen id : 9] +(14) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join condition: None -(16) Scan parquet default.web_sales +(15) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] - -(18) Filter [codegen id : 4] +(16) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Condition : isnotnull(ws_bill_customer_sk#12) -(19) ReusedExchange [Reuses operator id: 11] +(17) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -(20) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] Right keys [1]: [d_date_sk#7] Join condition: None -(21) Project [codegen id : 4] +(19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12] Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] -(22) BroadcastExchange +(20) BroadcastExchange Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(23) BroadcastHashJoin [codegen id : 9] +(21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#12] Join condition: None -(24) Scan parquet default.catalog_sales +(22) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] - -(26) Filter [codegen id : 6] +(23) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_ship_customer_sk#15) -(27) ReusedExchange [Reuses operator id: 11] +(24) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -(28) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#7] Join condition: None -(29) Project [codegen id : 6] +(26) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#15] Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] -(30) BroadcastExchange +(27) BroadcastExchange Input [1]: [cs_ship_customer_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] -(31) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#15] Join condition: None -(32) Project [codegen id : 9] +(29) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(33) Scan parquet default.customer_address +(30) Scan parquet default.customer_address Output [2]: [ca_address_sk#18, ca_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 7] +(31) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#18, ca_state#19] -(35) Filter [codegen id : 7] +(32) Filter [codegen id : 7] Input [2]: [ca_address_sk#18, ca_state#19] Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) -(36) Project [codegen id : 7] +(33) Project [codegen id : 7] Output [1]: [ca_address_sk#18] Input [2]: [ca_address_sk#18, ca_state#19] -(37) BroadcastExchange +(34) BroadcastExchange Input [1]: [ca_address_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] -(38) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#18] Join condition: None -(39) Project [codegen id : 9] +(36) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] -(40) Scan parquet default.customer_demographics +(37) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 8] +(38) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -(42) Filter [codegen id : 8] +(39) Filter [codegen id : 8] Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Condition : isnotnull(cd_demo_sk#21) -(43) BroadcastExchange +(40) BroadcastExchange Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(44) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#21] Join condition: None -(45) Project [codegen id : 9] +(42) Project [codegen id : 9] Output [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -(46) HashAggregate [codegen id : 9] +(43) HashAggregate [codegen id : 9] Input [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#28] Results [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] -(47) Exchange +(44) Exchange Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] Arguments: hashpartitioning(cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, 5), ENSURE_REQUIREMENTS, [id=#30] -(48) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 10] Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#31] Results [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, count(1)#31 AS cnt1#32, cd_purchase_estimate#25, count(1)#31 AS cnt2#33, cd_credit_rating#26, count(1)#31 AS cnt3#34] -(49) TakeOrderedAndProject +(46) TakeOrderedAndProject Input [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#26 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (50) +ReusedExchange (47) -(50) ReusedExchange [Reuses operator id: 11] +(47) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index 7f39a7ff1796f..957b9561b1752 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -22,12 +22,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,11 +40,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -53,11 +51,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index 1e8bb4e77fcb4..d31dbc3498ead 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -98,7 +98,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#8, s_county#9, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] +PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -106,7 +106,7 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_state#10)) +Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : ((isnotnull(ranking#19) AND (ranking#19 <= 5)) AND isnotnull(s_state#16)) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt index 8ee6469a58e56..7a5207362a5bd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #4 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk,s_state] + Filter [s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_county,s_state] @@ -44,7 +44,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (7) Project [s_state] - Filter [ranking,s_state] + Filter [ranking] InputAdapter Window [_w2,s_state] WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 3f90f0bcea49f..26fec145f4211 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -98,7 +98,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#8, s_county#9, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] +PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -106,7 +106,7 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_state#10)) +Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : ((isnotnull(ranking#19) AND (ranking#19 <= 5)) AND isnotnull(s_state#16)) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index d7b8e57cc8f4e..f52e7f9a11922 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #4 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk,s_state] + Filter [s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_county,s_state] @@ -44,7 +44,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (7) Project [s_state] - Filter [ranking,s_state] + Filter [ranking] InputAdapter Window [_w2,s_state] WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index 7d458b55d672d..d006b61d20c33 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -1,73 +1,72 @@ == Physical Plan == -TakeOrderedAndProject (62) -+- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) +TakeOrderedAndProject (61) ++- * Project (60) + +- * BroadcastHashJoin Inner BuildRight (59) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * HashAggregate (30) + : : +- Exchange (29) + : : +- * HashAggregate (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_returns (1) - : : : +- BroadcastExchange (20) - : : : +- * Project (19) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (18) + : : : +- BroadcastExchange (19) + : : : +- * Project (18) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (17) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (15) - : : : :- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (26) - : : +- * Filter (25) - : : +- * ColumnarToRow (24) - : : +- Scan parquet default.item (23) - : +- BroadcastExchange (44) - : +- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Filter (34) - : : : +- * ColumnarToRow (33) - : : : +- Scan parquet default.catalog_returns (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- BroadcastExchange (59) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Filter (49) - : : +- * ColumnarToRow (48) - : : +- Scan parquet default.web_returns (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (14) + : : : :- * ColumnarToRow (8) + : : : : +- Scan parquet default.date_dim (7) + : : : +- BroadcastExchange (13) + : : : +- * Project (12) + : : : +- * Filter (11) + : : : +- * ColumnarToRow (10) + : : : +- Scan parquet default.date_dim (9) + : : +- BroadcastExchange (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet default.item (22) + : +- BroadcastExchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * HashAggregate (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Filter (33) + : : : +- * ColumnarToRow (32) + : : : +- Scan parquet default.catalog_returns (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- BroadcastExchange (58) + +- * HashAggregate (57) + +- Exchange (56) + +- * HashAggregate (55) + +- * Project (54) + +- * BroadcastHashJoin Inner BuildRight (53) + :- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Filter (48) + : : +- * ColumnarToRow (47) + : : +- Scan parquet default.web_returns (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet default.store_returns Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(sr_item_sk)] ReadSchema: struct @@ -79,295 +78,270 @@ Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) (4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] +Output [2]: [d_date_sk#4, d_date#5] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#5, d_date#6] +Input [2]: [d_date_sk#4, d_date#5] (6) Filter [codegen id : 3] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#6)) +Input [2]: [d_date_sk#4, d_date#5] +Condition : isnotnull(d_date_sk#4) (7) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (8) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#6, d_week_seq#7] +Input [2]: [d_date#5, d_week_seq#6] -(9) Filter [codegen id : 2] -Input [2]: [d_date#6, d_week_seq#7] -Condition : (isnotnull(d_week_seq#7) AND isnotnull(d_date#6)) - -(10) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +(9) Scan parquet default.date_dim +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] +(10) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#5, d_week_seq#6] -(12) Filter [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] -Condition : (cast(d_date#6 as string) IN (2000-06-30,2000-09-27,2000-11-17) AND isnotnull(d_week_seq#7)) +(11) Filter [codegen id : 1] +Input [2]: [d_date#5, d_week_seq#6] +Condition : cast(d_date#5 as string) IN (2000-06-30,2000-09-27,2000-11-17) -(13) Project [codegen id : 1] -Output [1]: [d_week_seq#7 AS d_week_seq#7#8] -Input [2]: [d_date#6, d_week_seq#7] +(12) Project [codegen id : 1] +Output [1]: [d_week_seq#6 AS d_week_seq#6#7] +Input [2]: [d_date#5, d_week_seq#6] -(14) BroadcastExchange -Input [1]: [d_week_seq#7#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] +(13) BroadcastExchange +Input [1]: [d_week_seq#6#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] -(15) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#7] -Right keys [1]: [d_week_seq#7#8] +(14) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [d_week_seq#6] +Right keys [1]: [d_week_seq#6#7] Join condition: None -(16) Project [codegen id : 2] -Output [1]: [d_date#6 AS d_date#6#10] -Input [2]: [d_date#6, d_week_seq#7] +(15) Project [codegen id : 2] +Output [1]: [d_date#5 AS d_date#5#9] +Input [2]: [d_date#5, d_week_seq#6] -(17) BroadcastExchange -Input [1]: [d_date#6#10] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#11] +(16) BroadcastExchange +Input [1]: [d_date#5#9] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] -(18) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date#6] -Right keys [1]: [d_date#6#10] +(17) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date#5] +Right keys [1]: [d_date#5#9] Join condition: None -(19) Project [codegen id : 3] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +(18) Project [codegen id : 3] +Output [1]: [d_date_sk#4] +Input [2]: [d_date_sk#4, d_date#5] -(20) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +(19) BroadcastExchange +Input [1]: [d_date_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(21) BroadcastHashJoin [codegen id : 5] +(20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] -Right keys [1]: [cast(d_date_sk#5 as bigint)] +Right keys [1]: [cast(d_date_sk#4 as bigint)] Join condition: None -(22) Project [codegen id : 5] +(21) Project [codegen id : 5] Output [2]: [sr_item_sk#1, sr_return_quantity#2] -Input [4]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, d_date_sk#5] +Input [4]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, d_date_sk#4] -(23) Scan parquet default.item -Output [2]: [i_item_sk#13, i_item_id#14] +(22) Scan parquet default.item +Output [2]: [i_item_sk#12, i_item_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#13, i_item_id#14] +(23) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#12, i_item_id#13] -(25) Filter [codegen id : 4] -Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +(24) Filter [codegen id : 4] +Input [2]: [i_item_sk#12, i_item_id#13] +Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) -(26) BroadcastExchange -Input [2]: [i_item_sk#13, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] +(25) BroadcastExchange +Input [2]: [i_item_sk#12, i_item_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] -(27) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#1] -Right keys [1]: [cast(i_item_sk#13 as bigint)] +Right keys [1]: [cast(i_item_sk#12 as bigint)] Join condition: None -(28) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#14] -Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#13, i_item_id#14] +(27) Project [codegen id : 5] +Output [2]: [sr_return_quantity#2, i_item_id#13] +Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#12, i_item_id#13] -(29) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#2, i_item_id#14] -Keys [1]: [i_item_id#14] +(28) HashAggregate [codegen id : 5] +Input [2]: [sr_return_quantity#2, i_item_id#13] +Keys [1]: [i_item_id#13] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#14, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#13, sum#16] -(30) Exchange -Input [2]: [i_item_id#14, sum#17] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#18] +(29) Exchange +Input [2]: [i_item_id#13, sum#16] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] -(31) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#17] -Keys [1]: [i_item_id#14] +(30) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#13, sum#16] +Keys [1]: [i_item_id#13] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] -Results [2]: [i_item_id#14 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] +Results [2]: [i_item_id#13 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] -(32) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +(31) Scan parquet default.catalog_returns +Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(cr_returned_date_sk#24 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +(32) ColumnarToRow [codegen id : 10] +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] -(34) Filter [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] -Condition : isnotnull(cr_item_sk#22) +(33) Filter [codegen id : 10] +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Condition : isnotnull(cr_item_sk#21) -(35) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#5] +(34) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#4] -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#24] -Right keys [1]: [d_date_sk#5] +(35) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#23] +Right keys [1]: [d_date_sk#4] Join condition: None -(37) Project [codegen id : 10] -Output [2]: [cr_item_sk#22, cr_return_quantity#23] -Input [4]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, d_date_sk#5] +(36) Project [codegen id : 10] +Output [2]: [cr_item_sk#21, cr_return_quantity#22] +Input [4]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, d_date_sk#4] -(38) ReusedExchange [Reuses operator id: 26] -Output [2]: [i_item_sk#13, i_item_id#14] +(37) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#12, i_item_id#13] -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#22] -Right keys [1]: [i_item_sk#13] +(38) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_item_sk#21] +Right keys [1]: [i_item_sk#12] Join condition: None -(40) Project [codegen id : 10] -Output [2]: [cr_return_quantity#23, i_item_id#14] -Input [4]: [cr_item_sk#22, cr_return_quantity#23, i_item_sk#13, i_item_id#14] - -(41) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#23, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum#26] -Results [2]: [i_item_id#14, sum#27] - -(42) Exchange -Input [2]: [i_item_id#14, sum#27] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#28] - -(43) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#14, sum#27] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum(cr_return_quantity#23)#29] -Results [2]: [i_item_id#14 AS item_id#30, sum(cr_return_quantity#23)#29 AS cr_item_qty#31] - -(44) BroadcastExchange -Input [2]: [item_id#30, cr_item_qty#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] - -(45) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#30] +(39) Project [codegen id : 10] +Output [2]: [cr_return_quantity#22, i_item_id#13] +Input [4]: [cr_item_sk#21, cr_return_quantity#22, i_item_sk#12, i_item_id#13] + +(40) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#22, i_item_id#13] +Keys [1]: [i_item_id#13] +Functions [1]: [partial_sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum#24] +Results [2]: [i_item_id#13, sum#25] + +(41) Exchange +Input [2]: [i_item_id#13, sum#25] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#26] + +(42) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#13, sum#25] +Keys [1]: [i_item_id#13] +Functions [1]: [sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] +Results [2]: [i_item_id#13 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] + +(43) BroadcastExchange +Input [2]: [item_id#28, cr_item_qty#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] + +(44) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#28] Join condition: None -(46) Project [codegen id : 18] -Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#31] -Input [4]: [item_id#20, sr_item_qty#21, item_id#30, cr_item_qty#31] +(45) Project [codegen id : 18] +Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] +Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] -(47) Scan parquet default.web_returns -Output [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] +(46) Scan parquet default.web_returns +Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#35), dynamicpruningexpression(wr_returned_date_sk#35 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] +(47) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -(49) Filter [codegen id : 16] -Input [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] -Condition : isnotnull(wr_item_sk#33) +(48) Filter [codegen id : 16] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Condition : isnotnull(wr_item_sk#31) -(50) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#5] +(49) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#4] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#35] -Right keys [1]: [cast(d_date_sk#5 as bigint)] +(50) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#33] +Right keys [1]: [cast(d_date_sk#4 as bigint)] Join condition: None -(52) Project [codegen id : 16] -Output [2]: [wr_item_sk#33, wr_return_quantity#34] -Input [4]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35, d_date_sk#5] +(51) Project [codegen id : 16] +Output [2]: [wr_item_sk#31, wr_return_quantity#32] +Input [4]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, d_date_sk#4] -(53) ReusedExchange [Reuses operator id: 26] -Output [2]: [i_item_sk#13, i_item_id#14] +(52) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#12, i_item_id#13] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#33] -Right keys [1]: [cast(i_item_sk#13 as bigint)] +(53) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#31] +Right keys [1]: [cast(i_item_sk#12 as bigint)] Join condition: None -(55) Project [codegen id : 16] -Output [2]: [wr_return_quantity#34, i_item_id#14] -Input [4]: [wr_item_sk#33, wr_return_quantity#34, i_item_sk#13, i_item_id#14] - -(56) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#34, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(wr_return_quantity#34)] -Aggregate Attributes [1]: [sum#36] -Results [2]: [i_item_id#14, sum#37] - -(57) Exchange -Input [2]: [i_item_id#14, sum#37] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#38] - -(58) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#14, sum#37] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(wr_return_quantity#34)] -Aggregate Attributes [1]: [sum(wr_return_quantity#34)#39] -Results [2]: [i_item_id#14 AS item_id#40, sum(wr_return_quantity#34)#39 AS wr_item_qty#41] - -(59) BroadcastExchange -Input [2]: [item_id#40, wr_item_qty#41] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] - -(60) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#40] +(54) Project [codegen id : 16] +Output [2]: [wr_return_quantity#32, i_item_id#13] +Input [4]: [wr_item_sk#31, wr_return_quantity#32, i_item_sk#12, i_item_id#13] + +(55) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#32, i_item_id#13] +Keys [1]: [i_item_id#13] +Functions [1]: [partial_sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#13, sum#35] + +(56) Exchange +Input [2]: [i_item_id#13, sum#35] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#36] + +(57) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#13, sum#35] +Keys [1]: [i_item_id#13] +Functions [1]: [sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] +Results [2]: [i_item_id#13 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] + +(58) BroadcastExchange +Input [2]: [item_id#38, wr_item_qty#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] + +(59) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#38] Join condition: None -(61) Project [codegen id : 18] -Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS sr_dev#43, cr_item_qty#31, (((cast(cr_item_qty#31 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS cr_dev#44, wr_item_qty#41, (((cast(wr_item_qty#41 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS wr_dev#45, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#46] -Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#31, item_id#40, wr_item_qty#41] - -(62) TakeOrderedAndProject -Input [8]: [item_id#20, sr_item_qty#21, sr_dev#43, cr_item_qty#31, cr_dev#44, wr_item_qty#41, wr_dev#45, average#46] -Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#43, cr_item_qty#31, cr_dev#44, wr_item_qty#41, wr_dev#45, average#46] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -ReusedExchange (63) - - -(63) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#5] - -Subquery:2 Hosting operator id = 32 Hosting Expression = cr_returned_date_sk#24 IN dynamicpruning#25 -ReusedExchange (64) - - -(64) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#5] - -Subquery:3 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#35 IN dynamicpruning#4 +(60) Project [codegen id : 18] +Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] +Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] +(61) TakeOrderedAndProject +Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt index 237c969825d92..b46131afd5ec7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt @@ -17,14 +17,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #2 WholeStageCodegen (3) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] @@ -33,15 +31,14 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (2) Project [d_date] BroadcastHashJoin [d_week_seq,d_week_seq] - Filter [d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_week_seq] - Filter [d_date,d_week_seq] + Filter [d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] @@ -68,8 +65,6 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter @@ -90,7 +85,6 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index f33bf1d745219..c926ab7e3ccb6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,14 +1,14 @@ == Physical Plan == -TakeOrderedAndProject (62) -+- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) +TakeOrderedAndProject (61) ++- * Project (60) + +- * BroadcastHashJoin Inner BuildRight (59) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * HashAggregate (30) + : : +- Exchange (29) + : : +- * HashAggregate (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) : : :- * Project (9) : : : +- * BroadcastHashJoin Inner BuildRight (8) : : : :- * Filter (3) @@ -18,56 +18,55 @@ TakeOrderedAndProject (62) : : : +- * Filter (6) : : : +- * ColumnarToRow (5) : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin LeftSemi BuildRight (24) + : : +- BroadcastExchange (25) + : : +- * Project (24) + : : +- * BroadcastHashJoin LeftSemi BuildRight (23) : : :- * Filter (12) : : : +- * ColumnarToRow (11) : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin LeftSemi BuildRight (21) - : : :- * Filter (15) - : : : +- * ColumnarToRow (14) - : : : +- Scan parquet default.date_dim (13) - : : +- BroadcastExchange (20) - : : +- * Project (19) - : : +- * Filter (18) - : : +- * ColumnarToRow (17) - : : +- Scan parquet default.date_dim (16) - : +- BroadcastExchange (44) - : +- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Filter (34) - : : : +- * ColumnarToRow (33) - : : : +- Scan parquet default.catalog_returns (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- BroadcastExchange (59) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Filter (49) - : : +- * ColumnarToRow (48) - : : +- Scan parquet default.web_returns (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) + : : +- BroadcastExchange (22) + : : +- * Project (21) + : : +- * BroadcastHashJoin LeftSemi BuildRight (20) + : : :- * ColumnarToRow (14) + : : : +- Scan parquet default.date_dim (13) + : : +- BroadcastExchange (19) + : : +- * Project (18) + : : +- * Filter (17) + : : +- * ColumnarToRow (16) + : : +- Scan parquet default.date_dim (15) + : +- BroadcastExchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * HashAggregate (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Filter (33) + : : : +- * ColumnarToRow (32) + : : : +- Scan parquet default.catalog_returns (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- BroadcastExchange (58) + +- * HashAggregate (57) + +- Exchange (56) + +- * HashAggregate (55) + +- * Project (54) + +- * BroadcastHashJoin Inner BuildRight (53) + :- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Filter (48) + : : +- * ColumnarToRow (47) + : : +- Scan parquet default.web_returns (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet default.store_returns Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(sr_item_sk)] ReadSchema: struct @@ -79,295 +78,270 @@ Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) (4) Scan parquet default.item -Output [2]: [i_item_sk#5, i_item_id#6] +Output [2]: [i_item_sk#4, i_item_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#6] +Input [2]: [i_item_sk#4, i_item_id#5] (6) Filter [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) +Input [2]: [i_item_sk#4, i_item_id#5] +Condition : (isnotnull(i_item_sk#4) AND isnotnull(i_item_id#5)) (7) BroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] +Input [2]: [i_item_sk#4, i_item_id#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#6] (8) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#1] -Right keys [1]: [cast(i_item_sk#5 as bigint)] +Right keys [1]: [cast(i_item_sk#4 as bigint)] Join condition: None (9) Project [codegen id : 5] -Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] -Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#6] +Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#5] +Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#4, i_item_id#5] (10) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (11) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#7, d_date#8] (12) Filter [codegen id : 4] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (isnotnull(d_date_sk#8) AND isnotnull(d_date#9)) +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) (13) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [d_date#9, d_week_seq#10] +Input [2]: [d_date#8, d_week_seq#9] -(15) Filter [codegen id : 3] -Input [2]: [d_date#9, d_week_seq#10] -Condition : (isnotnull(d_week_seq#10) AND isnotnull(d_date#9)) - -(16) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +(15) Scan parquet default.date_dim +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#9, d_week_seq#10] +(16) ColumnarToRow [codegen id : 2] +Input [2]: [d_date#8, d_week_seq#9] -(18) Filter [codegen id : 2] -Input [2]: [d_date#9, d_week_seq#10] -Condition : (cast(d_date#9 as string) IN (2000-06-30,2000-09-27,2000-11-17) AND isnotnull(d_week_seq#10)) +(17) Filter [codegen id : 2] +Input [2]: [d_date#8, d_week_seq#9] +Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) -(19) Project [codegen id : 2] -Output [1]: [d_week_seq#10 AS d_week_seq#10#11] -Input [2]: [d_date#9, d_week_seq#10] +(18) Project [codegen id : 2] +Output [1]: [d_week_seq#9 AS d_week_seq#9#10] +Input [2]: [d_date#8, d_week_seq#9] -(20) BroadcastExchange -Input [1]: [d_week_seq#10#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +(19) BroadcastExchange +Input [1]: [d_week_seq#9#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(21) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#10#11] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_week_seq#9] +Right keys [1]: [d_week_seq#9#10] Join condition: None -(22) Project [codegen id : 3] -Output [1]: [d_date#9 AS d_date#9#13] -Input [2]: [d_date#9, d_week_seq#10] +(21) Project [codegen id : 3] +Output [1]: [d_date#8 AS d_date#8#12] +Input [2]: [d_date#8, d_week_seq#9] -(23) BroadcastExchange -Input [1]: [d_date#9#13] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#14] +(22) BroadcastExchange +Input [1]: [d_date#8#12] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [d_date#9] -Right keys [1]: [d_date#9#13] +(23) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [d_date#8] +Right keys [1]: [d_date#8#12] Join condition: None -(25) Project [codegen id : 4] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +(24) Project [codegen id : 4] +Output [1]: [d_date_sk#7] +Input [2]: [d_date_sk#7, d_date#8] -(26) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +(25) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(27) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] -Right keys [1]: [cast(d_date_sk#8 as bigint)] +Right keys [1]: [cast(d_date_sk#7 as bigint)] Join condition: None -(28) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#6] -Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6, d_date_sk#8] +(27) Project [codegen id : 5] +Output [2]: [sr_return_quantity#2, i_item_id#5] +Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#5, d_date_sk#7] -(29) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#2, i_item_id#6] -Keys [1]: [i_item_id#6] +(28) HashAggregate [codegen id : 5] +Input [2]: [sr_return_quantity#2, i_item_id#5] +Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#6, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#5, sum#16] -(30) Exchange -Input [2]: [i_item_id#6, sum#17] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#18] +(29) Exchange +Input [2]: [i_item_id#5, sum#16] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#17] -(31) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#6, sum#17] -Keys [1]: [i_item_id#6] +(30) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#5, sum#16] +Keys [1]: [i_item_id#5] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] -Results [2]: [i_item_id#6 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] +Results [2]: [i_item_id#5 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] -(32) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +(31) Scan parquet default.catalog_returns +Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(cr_returned_date_sk#24 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +(32) ColumnarToRow [codegen id : 10] +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] -(34) Filter [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] -Condition : isnotnull(cr_item_sk#22) +(33) Filter [codegen id : 10] +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Condition : isnotnull(cr_item_sk#21) -(35) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_item_id#6] +(34) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#4, i_item_id#5] -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#22] -Right keys [1]: [i_item_sk#5] +(35) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_item_sk#21] +Right keys [1]: [i_item_sk#4] Join condition: None -(37) Project [codegen id : 10] -Output [3]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#6] -Input [5]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, i_item_sk#5, i_item_id#6] +(36) Project [codegen id : 10] +Output [3]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5] +Input [5]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, i_item_sk#4, i_item_id#5] -(38) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#8] +(37) ReusedExchange [Reuses operator id: 25] +Output [1]: [d_date_sk#7] -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#24] -Right keys [1]: [d_date_sk#8] +(38) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#23] +Right keys [1]: [d_date_sk#7] Join condition: None -(40) Project [codegen id : 10] -Output [2]: [cr_return_quantity#23, i_item_id#6] -Input [4]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#6, d_date_sk#8] - -(41) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#23, i_item_id#6] -Keys [1]: [i_item_id#6] -Functions [1]: [partial_sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum#26] -Results [2]: [i_item_id#6, sum#27] - -(42) Exchange -Input [2]: [i_item_id#6, sum#27] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#28] - -(43) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#6, sum#27] -Keys [1]: [i_item_id#6] -Functions [1]: [sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum(cr_return_quantity#23)#29] -Results [2]: [i_item_id#6 AS item_id#30, sum(cr_return_quantity#23)#29 AS cr_item_qty#31] - -(44) BroadcastExchange -Input [2]: [item_id#30, cr_item_qty#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] - -(45) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#30] +(39) Project [codegen id : 10] +Output [2]: [cr_return_quantity#22, i_item_id#5] +Input [4]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5, d_date_sk#7] + +(40) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#22, i_item_id#5] +Keys [1]: [i_item_id#5] +Functions [1]: [partial_sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum#24] +Results [2]: [i_item_id#5, sum#25] + +(41) Exchange +Input [2]: [i_item_id#5, sum#25] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#26] + +(42) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#5, sum#25] +Keys [1]: [i_item_id#5] +Functions [1]: [sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] +Results [2]: [i_item_id#5 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] + +(43) BroadcastExchange +Input [2]: [item_id#28, cr_item_qty#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] + +(44) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#28] Join condition: None -(46) Project [codegen id : 18] -Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#31] -Input [4]: [item_id#20, sr_item_qty#21, item_id#30, cr_item_qty#31] +(45) Project [codegen id : 18] +Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] +Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] -(47) Scan parquet default.web_returns -Output [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] +(46) Scan parquet default.web_returns +Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#35), dynamicpruningexpression(wr_returned_date_sk#35 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] +(47) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -(49) Filter [codegen id : 16] -Input [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] -Condition : isnotnull(wr_item_sk#33) +(48) Filter [codegen id : 16] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Condition : isnotnull(wr_item_sk#31) -(50) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_item_id#6] +(49) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#4, i_item_id#5] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#33] -Right keys [1]: [cast(i_item_sk#5 as bigint)] +(50) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#31] +Right keys [1]: [cast(i_item_sk#4 as bigint)] Join condition: None -(52) Project [codegen id : 16] -Output [3]: [wr_return_quantity#34, wr_returned_date_sk#35, i_item_id#6] -Input [5]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35, i_item_sk#5, i_item_id#6] +(51) Project [codegen id : 16] +Output [3]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5] +Input [5]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, i_item_sk#4, i_item_id#5] -(53) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#8] +(52) ReusedExchange [Reuses operator id: 25] +Output [1]: [d_date_sk#7] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#35] -Right keys [1]: [cast(d_date_sk#8 as bigint)] +(53) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#33] +Right keys [1]: [cast(d_date_sk#7 as bigint)] Join condition: None -(55) Project [codegen id : 16] -Output [2]: [wr_return_quantity#34, i_item_id#6] -Input [4]: [wr_return_quantity#34, wr_returned_date_sk#35, i_item_id#6, d_date_sk#8] - -(56) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#34, i_item_id#6] -Keys [1]: [i_item_id#6] -Functions [1]: [partial_sum(wr_return_quantity#34)] -Aggregate Attributes [1]: [sum#36] -Results [2]: [i_item_id#6, sum#37] - -(57) Exchange -Input [2]: [i_item_id#6, sum#37] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#38] - -(58) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#6, sum#37] -Keys [1]: [i_item_id#6] -Functions [1]: [sum(wr_return_quantity#34)] -Aggregate Attributes [1]: [sum(wr_return_quantity#34)#39] -Results [2]: [i_item_id#6 AS item_id#40, sum(wr_return_quantity#34)#39 AS wr_item_qty#41] - -(59) BroadcastExchange -Input [2]: [item_id#40, wr_item_qty#41] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] - -(60) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#40] +(54) Project [codegen id : 16] +Output [2]: [wr_return_quantity#32, i_item_id#5] +Input [4]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5, d_date_sk#7] + +(55) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#32, i_item_id#5] +Keys [1]: [i_item_id#5] +Functions [1]: [partial_sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#5, sum#35] + +(56) Exchange +Input [2]: [i_item_id#5, sum#35] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#36] + +(57) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#5, sum#35] +Keys [1]: [i_item_id#5] +Functions [1]: [sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] +Results [2]: [i_item_id#5 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] + +(58) BroadcastExchange +Input [2]: [item_id#38, wr_item_qty#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] + +(59) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#38] Join condition: None -(61) Project [codegen id : 18] -Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS sr_dev#43, cr_item_qty#31, (((cast(cr_item_qty#31 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS cr_dev#44, wr_item_qty#41, (((cast(wr_item_qty#41 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS wr_dev#45, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#46] -Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#31, item_id#40, wr_item_qty#41] - -(62) TakeOrderedAndProject -Input [8]: [item_id#20, sr_item_qty#21, sr_dev#43, cr_item_qty#31, cr_dev#44, wr_item_qty#41, wr_dev#45, average#46] -Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#43, cr_item_qty#31, cr_dev#44, wr_item_qty#41, wr_dev#45, average#46] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -ReusedExchange (63) - - -(63) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#8] - -Subquery:2 Hosting operator id = 32 Hosting Expression = cr_returned_date_sk#24 IN dynamicpruning#25 -ReusedExchange (64) - - -(64) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#8] - -Subquery:3 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#35 IN dynamicpruning#4 +(60) Project [codegen id : 18] +Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] +Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] +(61) TakeOrderedAndProject +Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index fc2b465906e4e..7fc930f2885fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -17,21 +17,19 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #3 + BroadcastExchange #2 WholeStageCodegen (1) Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #2 + BroadcastExchange #3 WholeStageCodegen (4) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] @@ -40,15 +38,14 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (3) Project [d_date] BroadcastHashJoin [d_week_seq,d_week_seq] - Filter [d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) Project [d_week_seq] - Filter [d_date,d_week_seq] + Filter [d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] @@ -68,12 +65,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #3 + ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #8 WholeStageCodegen (17) @@ -90,8 +85,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #3 + ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt index 19cbce69c0768..2466a4d38f21e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * SortMergeJoin Inner (20) +TakeOrderedAndProject (24) ++- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * Project (20) + +- * SortMergeJoin Inner (19) :- * Sort (13) : +- Exchange (12) : +- * Project (11) @@ -18,12 +18,11 @@ TakeOrderedAndProject (25) : +- * Filter (7) : +- * ColumnarToRow (6) : +- Scan parquet default.reason (5) - +- * Sort (19) - +- Exchange (18) - +- * Project (17) - +- * Filter (16) - +- * ColumnarToRow (15) - +- Scan parquet default.store_sales (14) + +- * Sort (18) + +- Exchange (17) + +- * Project (16) + +- * ColumnarToRow (15) + +- Scan parquet default.store_sales (14) (1) Scan parquet default.store_returns @@ -87,56 +86,51 @@ Arguments: [sr_item_sk#1 ASC NULLS FIRST, sr_ticket_number#3 ASC NULLS FIRST], f Output [6]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14, ss_sold_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] Input [6]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14, ss_sold_date_sk#15] -(16) Filter [codegen id : 4] -Input [6]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14, ss_sold_date_sk#15] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_ticket_number#12)) - -(17) Project [codegen id : 4] +(16) Project [codegen id : 4] Output [5]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14] Input [6]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14, ss_sold_date_sk#15] -(18) Exchange +(17) Exchange Input [5]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14] Arguments: hashpartitioning(cast(ss_item_sk#10 as bigint), cast(ss_ticket_number#12 as bigint), 5), ENSURE_REQUIREMENTS, [id=#16] -(19) Sort [codegen id : 5] +(18) Sort [codegen id : 5] Input [5]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14] Arguments: [cast(ss_item_sk#10 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#12 as bigint) ASC NULLS FIRST], false, 0 -(20) SortMergeJoin [codegen id : 6] +(19) SortMergeJoin [codegen id : 6] Left keys [2]: [sr_item_sk#1, sr_ticket_number#3] Right keys [2]: [cast(ss_item_sk#10 as bigint), cast(ss_ticket_number#12 as bigint)] Join condition: None -(21) Project [codegen id : 6] +(20) Project [codegen id : 6] Output [2]: [ss_customer_sk#11, CASE WHEN isnotnull(sr_return_quantity#4) THEN CheckOverflow((promote_precision(cast(cast((ss_quantity#13 - sr_return_quantity#4) as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#14 as decimal(12,2)))), DecimalType(18,2), true) ELSE CheckOverflow((promote_precision(cast(cast(ss_quantity#13 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#14 as decimal(12,2)))), DecimalType(18,2), true) END AS act_sales#17] Input [8]: [sr_item_sk#1, sr_ticket_number#3, sr_return_quantity#4, ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14] -(22) HashAggregate [codegen id : 6] +(21) HashAggregate [codegen id : 6] Input [2]: [ss_customer_sk#11, act_sales#17] Keys [1]: [ss_customer_sk#11] Functions [1]: [partial_sum(act_sales#17)] Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [3]: [ss_customer_sk#11, sum#20, isEmpty#21] -(23) Exchange +(22) Exchange Input [3]: [ss_customer_sk#11, sum#20, isEmpty#21] Arguments: hashpartitioning(ss_customer_sk#11, 5), ENSURE_REQUIREMENTS, [id=#22] -(24) HashAggregate [codegen id : 7] +(23) HashAggregate [codegen id : 7] Input [3]: [ss_customer_sk#11, sum#20, isEmpty#21] Keys [1]: [ss_customer_sk#11] Functions [1]: [sum(act_sales#17)] Aggregate Attributes [1]: [sum(act_sales#17)#23] Results [2]: [ss_customer_sk#11, sum(act_sales#17)#23 AS sumsales#24] -(25) TakeOrderedAndProject +(24) TakeOrderedAndProject Input [2]: [ss_customer_sk#11, sumsales#24] Arguments: 100, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#11 ASC NULLS FIRST], [ss_customer_sk#11, sumsales#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt index fcebeb98855be..17464e295b1dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt @@ -35,7 +35,6 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] Exchange [ss_item_sk,ss_ticket_number] #4 WholeStageCodegen (4) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - Filter [ss_item_sk,ss_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index cd69276ebcb4a..1f3470198cd20 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -1,142 +1,136 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildRight (20) - :- * Project (14) - : +- * SortMergeJoin Inner (13) - : :- * Sort (6) - : : +- Exchange (5) - : : +- * Project (4) - : : +- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- * Sort (12) - : +- Exchange (11) - : +- * Project (10) - : +- * Filter (9) - : +- * ColumnarToRow (8) - : +- Scan parquet default.store_returns (7) - +- BroadcastExchange (19) - +- * Project (18) - +- * Filter (17) - +- * ColumnarToRow (16) - +- Scan parquet default.reason (15) +TakeOrderedAndProject (24) ++- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * Project (20) + +- * BroadcastHashJoin Inner BuildRight (19) + :- * Project (13) + : +- * SortMergeJoin Inner (12) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Project (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.store_sales (1) + : +- * Sort (11) + : +- Exchange (10) + : +- * Project (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.store_returns (6) + +- BroadcastExchange (18) + +- * Project (17) + +- * Filter (16) + +- * ColumnarToRow (15) + +- Scan parquet default.reason (14) (1) Scan parquet default.store_sales Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 1] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#3)) - -(4) Project [codegen id : 1] +(3) Project [codegen id : 1] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -(5) Exchange +(4) Exchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] Arguments: hashpartitioning(cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#3 as bigint), 5), ENSURE_REQUIREMENTS, [id=#7] -(6) Sort [codegen id : 2] +(5) Sort [codegen id : 2] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] Arguments: [cast(ss_item_sk#1 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#3 as bigint) ASC NULLS FIRST], false, 0 -(7) Scan parquet default.store_returns +(6) Scan parquet default.store_returns Output [5]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 3] +(7) ColumnarToRow [codegen id : 3] Input [5]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -(9) Filter [codegen id : 3] +(8) Filter [codegen id : 3] Input [5]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Condition : ((isnotnull(sr_item_sk#8) AND isnotnull(sr_ticket_number#10)) AND isnotnull(sr_reason_sk#9)) -(10) Project [codegen id : 3] +(9) Project [codegen id : 3] Output [4]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11] Input [5]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -(11) Exchange +(10) Exchange Input [4]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11] Arguments: hashpartitioning(sr_item_sk#8, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [id=#13] -(12) Sort [codegen id : 4] +(11) Sort [codegen id : 4] Input [4]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11] Arguments: [sr_item_sk#8 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin [codegen id : 6] +(12) SortMergeJoin [codegen id : 6] Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#3 as bigint)] Right keys [2]: [sr_item_sk#8, sr_ticket_number#10] Join condition: None -(14) Project [codegen id : 6] +(13) Project [codegen id : 6] Output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#9, sr_return_quantity#11] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11] -(15) Scan parquet default.reason +(14) Scan parquet default.reason Output [2]: [r_reason_sk#14, r_reason_desc#15] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 5] +(15) ColumnarToRow [codegen id : 5] Input [2]: [r_reason_sk#14, r_reason_desc#15] -(17) Filter [codegen id : 5] +(16) Filter [codegen id : 5] Input [2]: [r_reason_sk#14, r_reason_desc#15] Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28 )) AND isnotnull(r_reason_sk#14)) -(18) Project [codegen id : 5] +(17) Project [codegen id : 5] Output [1]: [r_reason_sk#14] Input [2]: [r_reason_sk#14, r_reason_desc#15] -(19) BroadcastExchange +(18) BroadcastExchange Input [1]: [r_reason_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(20) BroadcastHashJoin [codegen id : 6] +(19) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_reason_sk#9] Right keys [1]: [cast(r_reason_sk#14 as bigint)] Join condition: None -(21) Project [codegen id : 6] +(20) Project [codegen id : 6] Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#11) THEN CheckOverflow((promote_precision(cast(cast((ss_quantity#4 - sr_return_quantity#11) as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#5 as decimal(12,2)))), DecimalType(18,2), true) ELSE CheckOverflow((promote_precision(cast(cast(ss_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#5 as decimal(12,2)))), DecimalType(18,2), true) END AS act_sales#17] Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#9, sr_return_quantity#11, r_reason_sk#14] -(22) HashAggregate [codegen id : 6] +(21) HashAggregate [codegen id : 6] Input [2]: [ss_customer_sk#2, act_sales#17] Keys [1]: [ss_customer_sk#2] Functions [1]: [partial_sum(act_sales#17)] Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [3]: [ss_customer_sk#2, sum#20, isEmpty#21] -(23) Exchange +(22) Exchange Input [3]: [ss_customer_sk#2, sum#20, isEmpty#21] Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#22] -(24) HashAggregate [codegen id : 7] +(23) HashAggregate [codegen id : 7] Input [3]: [ss_customer_sk#2, sum#20, isEmpty#21] Keys [1]: [ss_customer_sk#2] Functions [1]: [sum(act_sales#17)] Aggregate Attributes [1]: [sum(act_sales#17)#23] Results [2]: [ss_customer_sk#2, sum(act_sales#17)#23 AS sumsales#24] -(25) TakeOrderedAndProject +(24) TakeOrderedAndProject Input [2]: [ss_customer_sk#2, sumsales#24] Arguments: 100, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index c6441ac0fc899..0d9aec90a2da4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -16,10 +16,9 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] Exchange [ss_item_sk,ss_ticket_number] #2 WholeStageCodegen (1) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - Filter [ss_item_sk,ss_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) Sort [sr_item_sk,sr_ticket_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index eb516affccfed..c77590bf71044 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -1,62 +1,60 @@ == Physical Plan == -* Sort (51) -+- * HashAggregate (50) - +- Exchange (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- SortMergeJoin LeftAnti (23) - : : : :- * Sort (16) - : : : : +- Exchange (15) - : : : : +- * Project (14) - : : : : +- SortMergeJoin LeftSemi (13) +* Sort (49) ++- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- SortMergeJoin LeftAnti (21) + : : : :- * Sort (15) + : : : : +- Exchange (14) + : : : : +- * Project (13) + : : : : +- SortMergeJoin LeftSemi (12) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.web_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.web_sales (7) - : : : +- * Sort (22) - : : : +- Exchange (21) - : : : +- * Project (20) - : : : +- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.web_returns (17) - : : +- BroadcastExchange (28) - : : +- * Project (27) - : : +- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.customer_address (24) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.web_site (31) - +- BroadcastExchange (42) - +- * Project (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.date_dim (38) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * Project (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.web_sales (7) + : : : +- * Sort (20) + : : : +- Exchange (19) + : : : +- * Project (18) + : : : +- * ColumnarToRow (17) + : : : +- Scan parquet default.web_returns (16) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet default.customer_address (22) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet default.web_site (29) + +- BroadcastExchange (40) + +- * Project (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet default.date_dim (36) (1) Scan parquet default.web_sales Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -64,7 +62,7 @@ Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (3) Filter [codegen id : 1] Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#5)) AND isnotnull(ws_warehouse_sk#4)) +Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) (4) Project [codegen id : 1] Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -82,206 +80,196 @@ Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -(9) Filter [codegen id : 3] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -Condition : (isnotnull(ws_order_number#5) AND isnotnull(ws_warehouse_sk#4)) - -(10) Project [codegen id : 3] +(9) Project [codegen id : 3] Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -(11) Exchange +(10) Exchange Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] -(12) Sort [codegen id : 4] +(11) Sort [codegen id : 4] Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin +(12) SortMergeJoin Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#5#11] Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) -(14) Project [codegen id : 5] +(13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(15) Exchange +(14) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] -(16) Sort [codegen id : 6] +(15) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 -(17) Scan parquet default.web_returns +(16) Scan parquet default.web_returns Output [2]: [wr_order_number#14, wr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 7] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] - -(19) Filter [codegen id : 7] +(17) ColumnarToRow [codegen id : 7] Input [2]: [wr_order_number#14, wr_returned_date_sk#15] -Condition : isnotnull(wr_order_number#14) -(20) Project [codegen id : 7] +(18) Project [codegen id : 7] Output [1]: [wr_order_number#14] Input [2]: [wr_order_number#14, wr_returned_date_sk#15] -(21) Exchange +(19) Exchange Input [1]: [wr_order_number#14] Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] -(22) Sort [codegen id : 8] +(20) Sort [codegen id : 8] Input [1]: [wr_order_number#14] Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 -(23) SortMergeJoin +(21) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] Right keys [1]: [wr_order_number#14] Join condition: None -(24) Scan parquet default.customer_address +(22) Scan parquet default.customer_address Output [2]: [ca_address_sk#17, ca_state#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 9] +(23) ColumnarToRow [codegen id : 9] Input [2]: [ca_address_sk#17, ca_state#18] -(26) Filter [codegen id : 9] +(24) Filter [codegen id : 9] Input [2]: [ca_address_sk#17, ca_state#18] Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = IL)) AND isnotnull(ca_address_sk#17)) -(27) Project [codegen id : 9] +(25) Project [codegen id : 9] Output [1]: [ca_address_sk#17] Input [2]: [ca_address_sk#17, ca_state#18] -(28) BroadcastExchange +(26) BroadcastExchange Input [1]: [ca_address_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(29) BroadcastHashJoin [codegen id : 12] +(27) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#17] Join condition: None -(30) Project [codegen id : 12] +(28) Project [codegen id : 12] Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#17] -(31) Scan parquet default.web_site +(29) Scan parquet default.web_site Output [2]: [web_site_sk#20, web_company_name#21] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] +(30) ColumnarToRow [codegen id : 10] Input [2]: [web_site_sk#20, web_company_name#21] -(33) Filter [codegen id : 10] +(31) Filter [codegen id : 10] Input [2]: [web_site_sk#20, web_company_name#21] Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri )) AND isnotnull(web_site_sk#20)) -(34) Project [codegen id : 10] +(32) Project [codegen id : 10] Output [1]: [web_site_sk#20] Input [2]: [web_site_sk#20, web_company_name#21] -(35) BroadcastExchange +(33) BroadcastExchange Input [1]: [web_site_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(36) BroadcastHashJoin [codegen id : 12] +(34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#20] Join condition: None -(37) Project [codegen id : 12] +(35) Project [codegen id : 12] Output [4]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#20] -(38) Scan parquet default.date_dim +(36) Scan parquet default.date_dim Output [2]: [d_date_sk#23, d_date#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 11] +(37) ColumnarToRow [codegen id : 11] Input [2]: [d_date_sk#23, d_date#24] -(40) Filter [codegen id : 11] +(38) Filter [codegen id : 11] Input [2]: [d_date_sk#23, d_date#24] Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 1999-02-01)) AND (d_date#24 <= 1999-04-02)) AND isnotnull(d_date_sk#23)) -(41) Project [codegen id : 11] +(39) Project [codegen id : 11] Output [1]: [d_date_sk#23] Input [2]: [d_date_sk#23, d_date#24] -(42) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] -(43) BroadcastHashJoin [codegen id : 12] +(41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#23] Join condition: None -(44) Project [codegen id : 12] +(42) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#23] -(45) HashAggregate [codegen id : 12] +(43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(46) Exchange +(44) Exchange Input [3]: [ws_order_number#5, sum#28, sum#29] Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] -(47) HashAggregate [codegen id : 13] +(45) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(48) HashAggregate [codegen id : 13] +(46) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] Results [3]: [sum#28, sum#29, count#32] -(49) Exchange +(47) Exchange Input [3]: [sum#28, sum#29, count#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] -(50) HashAggregate [codegen id : 14] +(48) HashAggregate [codegen id : 14] Input [3]: [sum#28, sum#29, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] -(51) Sort [codegen id : 14] +(49) Sort [codegen id : 14] Input [3]: [order count #34, total shipping cost #35, total net profit #36] Arguments: [order count #34 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt index 6a7459768a8fd..3ef7977c4bc64 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt @@ -32,7 +32,7 @@ WholeStageCodegen (14) Exchange [ws_order_number] #4 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_warehouse_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] @@ -42,20 +42,18 @@ WholeStageCodegen (14) Exchange [ws_order_number] #5 WholeStageCodegen (3) Project [ws_warehouse_sk,ws_order_number] - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] WholeStageCodegen (8) Sort [wr_order_number] InputAdapter Exchange [wr_order_number] #6 WholeStageCodegen (7) Project [wr_order_number] - Filter [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter BroadcastExchange #7 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index e6001eae60e57..9558a01423452 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -1,62 +1,60 @@ == Physical Plan == -* Sort (51) -+- * HashAggregate (50) - +- Exchange (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- SortMergeJoin LeftAnti (23) - : : : :- * Sort (16) - : : : : +- Exchange (15) - : : : : +- * Project (14) - : : : : +- SortMergeJoin LeftSemi (13) +* Sort (49) ++- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- SortMergeJoin LeftAnti (21) + : : : :- * Sort (15) + : : : : +- Exchange (14) + : : : : +- * Project (13) + : : : : +- SortMergeJoin LeftSemi (12) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.web_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.web_sales (7) - : : : +- * Sort (22) - : : : +- Exchange (21) - : : : +- * Project (20) - : : : +- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.web_returns (17) - : : +- BroadcastExchange (28) - : : +- * Project (27) - : : +- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.date_dim (24) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.customer_address (31) - +- BroadcastExchange (42) - +- * Project (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.web_site (38) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * Project (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.web_sales (7) + : : : +- * Sort (20) + : : : +- Exchange (19) + : : : +- * Project (18) + : : : +- * ColumnarToRow (17) + : : : +- Scan parquet default.web_returns (16) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet default.date_dim (22) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet default.customer_address (29) + +- BroadcastExchange (40) + +- * Project (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet default.web_site (36) (1) Scan parquet default.web_sales Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -64,7 +62,7 @@ Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (3) Filter [codegen id : 1] Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#5)) AND isnotnull(ws_warehouse_sk#4)) +Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) (4) Project [codegen id : 1] Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -82,206 +80,196 @@ Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -(9) Filter [codegen id : 3] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -Condition : (isnotnull(ws_order_number#5) AND isnotnull(ws_warehouse_sk#4)) - -(10) Project [codegen id : 3] +(9) Project [codegen id : 3] Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -(11) Exchange +(10) Exchange Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] -(12) Sort [codegen id : 4] +(11) Sort [codegen id : 4] Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin +(12) SortMergeJoin Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#5#11] Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) -(14) Project [codegen id : 5] +(13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(15) Exchange +(14) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] -(16) Sort [codegen id : 6] +(15) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 -(17) Scan parquet default.web_returns +(16) Scan parquet default.web_returns Output [2]: [wr_order_number#14, wr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 7] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] - -(19) Filter [codegen id : 7] +(17) ColumnarToRow [codegen id : 7] Input [2]: [wr_order_number#14, wr_returned_date_sk#15] -Condition : isnotnull(wr_order_number#14) -(20) Project [codegen id : 7] +(18) Project [codegen id : 7] Output [1]: [wr_order_number#14] Input [2]: [wr_order_number#14, wr_returned_date_sk#15] -(21) Exchange +(19) Exchange Input [1]: [wr_order_number#14] Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] -(22) Sort [codegen id : 8] +(20) Sort [codegen id : 8] Input [1]: [wr_order_number#14] Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 -(23) SortMergeJoin +(21) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] Right keys [1]: [wr_order_number#14] Join condition: None -(24) Scan parquet default.date_dim +(22) Scan parquet default.date_dim Output [2]: [d_date_sk#17, d_date#18] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 9] +(23) ColumnarToRow [codegen id : 9] Input [2]: [d_date_sk#17, d_date#18] -(26) Filter [codegen id : 9] +(24) Filter [codegen id : 9] Input [2]: [d_date_sk#17, d_date#18] Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-01)) AND (d_date#18 <= 1999-04-02)) AND isnotnull(d_date_sk#17)) -(27) Project [codegen id : 9] +(25) Project [codegen id : 9] Output [1]: [d_date_sk#17] Input [2]: [d_date_sk#17, d_date#18] -(28) BroadcastExchange +(26) BroadcastExchange Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(29) BroadcastHashJoin [codegen id : 12] +(27) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#17] Join condition: None -(30) Project [codegen id : 12] +(28) Project [codegen id : 12] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#17] -(31) Scan parquet default.customer_address +(29) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] +(30) ColumnarToRow [codegen id : 10] Input [2]: [ca_address_sk#20, ca_state#21] -(33) Filter [codegen id : 10] +(31) Filter [codegen id : 10] Input [2]: [ca_address_sk#20, ca_state#21] Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = IL)) AND isnotnull(ca_address_sk#20)) -(34) Project [codegen id : 10] +(32) Project [codegen id : 10] Output [1]: [ca_address_sk#20] Input [2]: [ca_address_sk#20, ca_state#21] -(35) BroadcastExchange +(33) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(36) BroadcastHashJoin [codegen id : 12] +(34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#20] Join condition: None -(37) Project [codegen id : 12] +(35) Project [codegen id : 12] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#20] -(38) Scan parquet default.web_site +(36) Scan parquet default.web_site Output [2]: [web_site_sk#23, web_company_name#24] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 11] +(37) ColumnarToRow [codegen id : 11] Input [2]: [web_site_sk#23, web_company_name#24] -(40) Filter [codegen id : 11] +(38) Filter [codegen id : 11] Input [2]: [web_site_sk#23, web_company_name#24] Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri )) AND isnotnull(web_site_sk#23)) -(41) Project [codegen id : 11] +(39) Project [codegen id : 11] Output [1]: [web_site_sk#23] Input [2]: [web_site_sk#23, web_company_name#24] -(42) BroadcastExchange +(40) BroadcastExchange Input [1]: [web_site_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] -(43) BroadcastHashJoin [codegen id : 12] +(41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#23] Join condition: None -(44) Project [codegen id : 12] +(42) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#23] -(45) HashAggregate [codegen id : 12] +(43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(46) Exchange +(44) Exchange Input [3]: [ws_order_number#5, sum#28, sum#29] Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] -(47) HashAggregate [codegen id : 13] +(45) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(48) HashAggregate [codegen id : 13] +(46) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] Results [3]: [sum#28, sum#29, count#32] -(49) Exchange +(47) Exchange Input [3]: [sum#28, sum#29, count#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] -(50) HashAggregate [codegen id : 14] +(48) HashAggregate [codegen id : 14] Input [3]: [sum#28, sum#29, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] -(51) Sort [codegen id : 14] +(49) Sort [codegen id : 14] Input [3]: [order count #34, total shipping cost #35, total net profit #36] Arguments: [order count #34 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 1282ca0d0f702..6bcd2aa24ccbb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -32,7 +32,7 @@ WholeStageCodegen (14) Exchange [ws_order_number] #4 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_warehouse_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] @@ -42,20 +42,18 @@ WholeStageCodegen (14) Exchange [ws_order_number] #5 WholeStageCodegen (3) Project [ws_warehouse_sk,ws_order_number] - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] WholeStageCodegen (8) Sort [wr_order_number] InputAdapter Exchange [wr_order_number] #6 WholeStageCodegen (7) Project [wr_order_number] - Filter [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter BroadcastExchange #7 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt index d111152b95384..320a93e19bb27 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt @@ -69,7 +69,7 @@ Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -77,7 +77,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num (3) Filter [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : (((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#4)) +Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) (4) Project [codegen id : 1] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt index 53b0ccfc3e218..439f3935c65a3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt @@ -29,7 +29,7 @@ WholeStageCodegen (23) Exchange [ws_order_number] #4 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index f57897991b8fa..2f719a4be1630 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -69,7 +69,7 @@ Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -77,7 +77,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num (3) Filter [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : (((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#4)) +Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) (4) Project [codegen id : 1] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 16bb4ac84f00c..adf95b02f07e5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -29,7 +29,7 @@ WholeStageCodegen (23) Exchange [ws_order_number] #4 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index 422a89af349bc..58d28b28dae10 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -1,63 +1,60 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- * Project (48) - +- * BroadcastHashJoin Inner BuildLeft (47) - :- BroadcastExchange (43) - : +- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (35) - : : +- SortMergeJoin LeftSemi (34) - : : :- SortMergeJoin LeftSemi (18) +TakeOrderedAndProject (49) ++- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * Project (45) + +- * BroadcastHashJoin Inner BuildLeft (44) + :- BroadcastExchange (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (32) + : : +- SortMergeJoin LeftSemi (31) + : : :- SortMergeJoin LeftSemi (17) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.customer (1) - : : : +- * Sort (17) - : : : +- Exchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * Filter (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.store_sales (6) - : : : +- BroadcastExchange (13) - : : : +- * Project (12) - : : : +- * Filter (11) - : : : +- * ColumnarToRow (10) - : : : +- Scan parquet default.date_dim (9) - : : +- * Sort (33) - : : +- Exchange (32) - : : +- Union (31) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (21) - : : : : +- * ColumnarToRow (20) - : : : : +- Scan parquet default.web_sales (19) - : : : +- ReusedExchange (22) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet default.catalog_sales (25) - : : +- ReusedExchange (28) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- * Filter (38) - : +- * ColumnarToRow (37) - : +- Scan parquet default.customer_address (36) - +- * Filter (46) - +- * ColumnarToRow (45) - +- Scan parquet default.customer_demographics (44) + : : : +- * Sort (16) + : : : +- Exchange (15) + : : : +- * Project (14) + : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : :- * ColumnarToRow (7) + : : : : +- Scan parquet default.store_sales (6) + : : : +- BroadcastExchange (12) + : : : +- * Project (11) + : : : +- * Filter (10) + : : : +- * ColumnarToRow (9) + : : : +- Scan parquet default.date_dim (8) + : : +- * Sort (30) + : : +- Exchange (29) + : : +- Union (28) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- Scan parquet default.web_sales (18) + : : : +- ReusedExchange (20) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * ColumnarToRow (24) + : : : +- Scan parquet default.catalog_sales (23) + : : +- ReusedExchange (25) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- * Filter (35) + : +- * ColumnarToRow (34) + : +- Scan parquet default.customer_address (33) + +- * Filter (43) + +- * ColumnarToRow (42) + +- Scan parquet default.customer_demographics (41) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -65,7 +62,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -80,224 +77,209 @@ Output [2]: [ss_customer_sk#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -(8) Filter [codegen id : 4] -Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_customer_sk#5) - -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#8, d_year#9, d_moy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] Condition : (((((isnotnull(d_year#9) AND isnotnull(d_moy#10)) AND (d_year#9 = 2002)) AND (d_moy#10 >= 4)) AND (d_moy#10 <= 7)) AND isnotnull(d_date_sk#8)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [1]: [d_date_sk#8] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -(13) BroadcastExchange +(12) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#8] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [1]: [ss_customer_sk#5] Input [3]: [ss_customer_sk#5, ss_sold_date_sk#6, d_date_sk#8] -(16) Exchange +(15) Exchange Input [1]: [ss_customer_sk#5] Arguments: hashpartitioning(ss_customer_sk#5, 5), ENSURE_REQUIREMENTS, [id=#12] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [1]: [ss_customer_sk#5] Arguments: [ss_customer_sk#5 ASC NULLS FIRST], false, 0 -(18) SortMergeJoin +(17) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#5] Join condition: None -(19) Scan parquet default.web_sales +(18) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#14), dynamicpruningexpression(ws_sold_date_sk#14 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 7] -Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] - -(21) Filter [codegen id : 7] +(19) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] -Condition : isnotnull(ws_bill_customer_sk#13) -(22) ReusedExchange [Reuses operator id: 13] +(20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -(23) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] Right keys [1]: [d_date_sk#8] Join condition: None -(24) Project [codegen id : 7] +(22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13 AS customer_sk#15] Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] -(25) Scan parquet default.catalog_sales +(23) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 9] +(24) ColumnarToRow [codegen id : 9] Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -(27) Filter [codegen id : 9] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Condition : isnotnull(cs_ship_customer_sk#16) - -(28) ReusedExchange [Reuses operator id: 13] +(25) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -(29) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#8] Join condition: None -(30) Project [codegen id : 9] +(27) Project [codegen id : 9] Output [1]: [cs_ship_customer_sk#16 AS customer_sk#18] Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] -(31) Union +(28) Union -(32) Exchange +(29) Exchange Input [1]: [customer_sk#15] Arguments: hashpartitioning(customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] -(33) Sort [codegen id : 10] +(30) Sort [codegen id : 10] Input [1]: [customer_sk#15] Arguments: [customer_sk#15 ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(31) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#15] Join condition: None -(35) Project [codegen id : 12] +(32) Project [codegen id : 12] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(36) Scan parquet default.customer_address +(33) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 11] +(34) ColumnarToRow [codegen id : 11] Input [2]: [ca_address_sk#20, ca_county#21] -(38) Filter [codegen id : 11] +(35) Filter [codegen id : 11] Input [2]: [ca_address_sk#20, ca_county#21] Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) -(39) Project [codegen id : 11] +(36) Project [codegen id : 11] Output [1]: [ca_address_sk#20] Input [2]: [ca_address_sk#20, ca_county#21] -(40) BroadcastExchange +(37) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(41) BroadcastHashJoin [codegen id : 12] +(38) BroadcastHashJoin [codegen id : 12] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#20] Join condition: None -(42) Project [codegen id : 12] +(39) Project [codegen id : 12] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] -(43) BroadcastExchange +(40) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] -(44) Scan parquet default.customer_demographics +(41) Scan parquet default.customer_demographics Output [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(45) ColumnarToRow +(42) ColumnarToRow Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -(46) Filter +(43) Filter Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Condition : isnotnull(cd_demo_sk#24) -(47) BroadcastHashJoin [codegen id : 13] +(44) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#24] Join condition: None -(48) Project [codegen id : 13] +(45) Project [codegen id : 13] Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -(49) HashAggregate [codegen id : 13] +(46) HashAggregate [codegen id : 13] Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#33] Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] -(50) Exchange +(47) Exchange Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#35] -(51) HashAggregate [codegen id : 14] +(48) HashAggregate [codegen id : 14] Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#36] Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#36 AS cnt1#37, cd_purchase_estimate#28, count(1)#36 AS cnt2#38, cd_credit_rating#29, count(1)#36 AS cnt3#39, cd_dep_count#30, count(1)#36 AS cnt4#40, cd_dep_employed_count#31, count(1)#36 AS cnt5#41, cd_dep_college_count#32, count(1)#36 AS cnt6#42] -(52) TakeOrderedAndProject +(49) TakeOrderedAndProject Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -ReusedExchange (53) +ReusedExchange (50) -(53) ReusedExchange [Reuses operator id: 13] +(50) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt index 1d4fd5a709520..220059a6f5e27 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -32,12 +32,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #5 WholeStageCodegen (3) @@ -54,21 +53,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (7) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (9) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index 1a258296802b5..eac6acc3543ca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -1,59 +1,56 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (31) - : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : :- * BroadcastHashJoin LeftSemi BuildRight (15) +TakeOrderedAndProject (45) ++- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin LeftSemi BuildRight (27) + : : :- * BroadcastHashJoin LeftSemi BuildRight (14) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.customer (1) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.store_sales (4) - : : : +- BroadcastExchange (11) - : : : +- * Project (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.date_dim (7) - : : +- BroadcastExchange (29) - : : +- Union (28) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (19) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.customer_address (32) - +- BroadcastExchange (42) - +- * Filter (41) - +- * ColumnarToRow (40) - +- Scan parquet default.customer_demographics (39) + : : : +- BroadcastExchange (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * ColumnarToRow (5) + : : : : +- Scan parquet default.store_sales (4) + : : : +- BroadcastExchange (10) + : : : +- * Project (9) + : : : +- * Filter (8) + : : : +- * ColumnarToRow (7) + : : : +- Scan parquet default.date_dim (6) + : : +- BroadcastExchange (26) + : : +- Union (25) + : : :- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * ColumnarToRow (21) + : : : +- Scan parquet default.catalog_sales (20) + : : +- ReusedExchange (22) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet default.customer_address (29) + +- BroadcastExchange (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet default.customer_demographics (36) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -61,223 +58,208 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) Filter [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Condition : isnotnull(ss_customer_sk#4) - -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [1]: [d_date_sk#7] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -(11) BroadcastExchange +(10) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] -(12) BroadcastHashJoin [codegen id : 2] +(11) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join condition: None -(13) Project [codegen id : 2] +(12) Project [codegen id : 2] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -(14) BroadcastExchange +(13) BroadcastExchange Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(15) BroadcastHashJoin [codegen id : 9] +(14) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join condition: None -(16) Scan parquet default.web_sales +(15) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] - -(18) Filter [codegen id : 4] +(16) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Condition : isnotnull(ws_bill_customer_sk#12) -(19) ReusedExchange [Reuses operator id: 11] +(17) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -(20) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] Right keys [1]: [d_date_sk#7] Join condition: None -(21) Project [codegen id : 4] +(19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12 AS customer_sk#14] Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] -(22) Scan parquet default.catalog_sales +(20) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] - -(24) Filter [codegen id : 6] +(21) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_ship_customer_sk#15) -(25) ReusedExchange [Reuses operator id: 11] +(22) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -(26) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#7] Join condition: None -(27) Project [codegen id : 6] +(24) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#15 AS customer_sk#17] Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] -(28) Union +(25) Union -(29) BroadcastExchange +(26) BroadcastExchange Input [1]: [customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(30) BroadcastHashJoin [codegen id : 9] +(27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#14] Join condition: None -(31) Project [codegen id : 9] +(28) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(32) Scan parquet default.customer_address +(29) Scan parquet default.customer_address Output [2]: [ca_address_sk#19, ca_county#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 7] +(30) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#19, ca_county#20] -(34) Filter [codegen id : 7] +(31) Filter [codegen id : 7] Input [2]: [ca_address_sk#19, ca_county#20] Condition : (ca_county#20 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#19)) -(35) Project [codegen id : 7] +(32) Project [codegen id : 7] Output [1]: [ca_address_sk#19] Input [2]: [ca_address_sk#19, ca_county#20] -(36) BroadcastExchange +(33) BroadcastExchange Input [1]: [ca_address_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(37) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#19] Join condition: None -(38) Project [codegen id : 9] +(35) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] -(39) Scan parquet default.customer_demographics +(36) Scan parquet default.customer_demographics Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(40) ColumnarToRow [codegen id : 8] +(37) ColumnarToRow [codegen id : 8] Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(41) Filter [codegen id : 8] +(38) Filter [codegen id : 8] Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Condition : isnotnull(cd_demo_sk#22) -(42) BroadcastExchange +(39) BroadcastExchange Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] -(43) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#22] Join condition: None -(44) Project [codegen id : 9] +(41) Project [codegen id : 9] Output [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(45) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 9] Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#32] Results [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] -(46) Exchange +(43) Exchange Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#34] -(47) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 10] Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#35] Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#35 AS cnt1#36, cd_purchase_estimate#26, count(1)#35 AS cnt2#37, cd_credit_rating#27, count(1)#35 AS cnt3#38, cd_dep_count#28, count(1)#35 AS cnt4#39, cd_dep_employed_count#29, count(1)#35 AS cnt5#40, cd_dep_college_count#30, count(1)#35 AS cnt6#41] -(48) TakeOrderedAndProject +(45) TakeOrderedAndProject Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (49) +ReusedExchange (46) -(49) ReusedExchange [Reuses operator id: 11] +(46) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 72526cd0b007a..5077b91e30c1b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -21,12 +21,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,21 +40,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter 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 f100c3e7e6ebf..312aba94bd9c6 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 @@ -144,7 +144,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (7) ColumnarToRow [codegen id : 20] @@ -152,7 +152,7 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] 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 be55d3b2fecfc..1d5907e319658 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 @@ -75,7 +75,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (20) 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,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index b4ab1eb60e90f..258224e574cfb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -120,7 +120,7 @@ Condition : isnotnull(ss_item_sk#1) 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), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -128,7 +128,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] 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)) AND isnotnull(i_item_sk#6)) +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 592d91ddf08fd..3507d76bcf9b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -67,7 +67,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] 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 b8f60c510269a..9b47400d3b780 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 @@ -241,7 +241,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (7) ColumnarToRow [codegen id : 20] @@ -249,7 +249,7 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] 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 d418fe48a4d67..d27eb5a32b387 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 @@ -97,7 +97,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (20) 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,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index c7b4a326a053f..04fe14cab2c9b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -214,7 +214,7 @@ Condition : isnotnull(ss_item_sk#1) 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), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -222,7 +222,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] 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)) AND isnotnull(i_item_sk#6)) +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 80c58cd303dc5..1634c1e247a12 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -89,7 +89,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt index 9657cdfd07984..0a861f4ce8e6d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt @@ -1,69 +1,68 @@ == Physical Plan == -TakeOrderedAndProject (58) -+- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * SortMergeJoin Inner (53) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (38) - : : +- Exchange (37) - : : +- * Project (36) - : : +- * Filter (35) - : : +- SortMergeJoin ExistenceJoin(exists#1) (34) - : : :- SortMergeJoin ExistenceJoin(exists#2) (26) - : : : :- SortMergeJoin LeftSemi (18) +TakeOrderedAndProject (57) ++- * HashAggregate (56) + +- Exchange (55) + +- * HashAggregate (54) + +- * Project (53) + +- * SortMergeJoin Inner (52) + :- * Sort (46) + : +- Exchange (45) + : +- * Project (44) + : +- * SortMergeJoin Inner (43) + : :- * Sort (37) + : : +- Exchange (36) + : : +- * Project (35) + : : +- * Filter (34) + : : +- SortMergeJoin ExistenceJoin(exists#1) (33) + : : :- SortMergeJoin ExistenceJoin(exists#2) (25) + : : : :- SortMergeJoin LeftSemi (17) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- * Sort (17) - : : : : +- Exchange (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Filter (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- Scan parquet default.store_sales (6) - : : : : +- BroadcastExchange (13) - : : : : +- * Project (12) - : : : : +- * Filter (11) - : : : : +- * ColumnarToRow (10) - : : : : +- Scan parquet default.date_dim (9) - : : : +- * Sort (25) - : : : +- Exchange (24) - : : : +- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * ColumnarToRow (20) - : : : : +- Scan parquet default.web_sales (19) - : : : +- ReusedExchange (21) - : : +- * Sort (33) - : : +- Exchange (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- Scan parquet default.catalog_sales (27) - : : +- ReusedExchange (29) - : +- * Sort (43) - : +- Exchange (42) - : +- * Filter (41) - : +- * ColumnarToRow (40) - : +- Scan parquet default.customer_address (39) - +- * Sort (52) - +- Exchange (51) - +- * Filter (50) - +- * ColumnarToRow (49) - +- Scan parquet default.customer_demographics (48) + : : : : +- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Project (14) + : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : :- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (12) + : : : : +- * Project (11) + : : : : +- * Filter (10) + : : : : +- * ColumnarToRow (9) + : : : : +- Scan parquet default.date_dim (8) + : : : +- * Sort (24) + : : : +- Exchange (23) + : : : +- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- Scan parquet default.web_sales (18) + : : : +- ReusedExchange (20) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * ColumnarToRow (27) + : : : +- Scan parquet default.catalog_sales (26) + : : +- ReusedExchange (28) + : +- * Sort (42) + : +- Exchange (41) + : +- * Filter (40) + : +- * ColumnarToRow (39) + : +- Scan parquet default.customer_address (38) + +- * Sort (51) + +- Exchange (50) + +- * Filter (49) + +- * ColumnarToRow (48) + +- Scan parquet default.customer_demographics (47) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -71,7 +70,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -86,249 +85,244 @@ Output [2]: [ss_customer_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -(8) Filter [codegen id : 4] -Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_customer_sk#7) - -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_qoy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_qoy#12)) AND (d_year#11 = 2002)) AND (d_qoy#12 < 4)) AND isnotnull(d_date_sk#10)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] -(13) BroadcastExchange +(12) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [1]: [ss_customer_sk#7] Input [3]: [ss_customer_sk#7, ss_sold_date_sk#8, d_date_sk#10] -(16) Exchange +(15) Exchange Input [1]: [ss_customer_sk#7] Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [1]: [ss_customer_sk#7] Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(18) SortMergeJoin +(17) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#7] Join condition: None -(19) Scan parquet default.web_sales +(18) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#16), dynamicpruningexpression(ws_sold_date_sk#16 IN dynamicpruning#9)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] -(21) ReusedExchange [Reuses operator id: 13] +(20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -(22) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(23) Project [codegen id : 7] +(22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] -(24) Exchange +(23) Exchange Input [1]: [ws_bill_customer_sk#15] Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] -(25) Sort [codegen id : 8] +(24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] Arguments: [ws_bill_customer_sk#15 ASC NULLS FIRST], false, 0 -(26) SortMergeJoin +(25) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#15] Join condition: None -(27) Scan parquet default.catalog_sales +(26) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 10] +(27) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] -(29) ReusedExchange [Reuses operator id: 13] +(28) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -(30) BroadcastHashJoin [codegen id : 10] +(29) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#19] Right keys [1]: [d_date_sk#10] Join condition: None -(31) Project [codegen id : 10] +(30) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#18] Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] -(32) Exchange +(31) Exchange Input [1]: [cs_ship_customer_sk#18] Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] -(33) Sort [codegen id : 11] +(32) Sort [codegen id : 11] Input [1]: [cs_ship_customer_sk#18] Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#18] Join condition: None -(35) Filter [codegen id : 12] +(34) Filter [codegen id : 12] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(36) Project [codegen id : 12] +(35) Project [codegen id : 12] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(37) Exchange +(36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] -(38) Sort [codegen id : 13] +(37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 -(39) Scan parquet default.customer_address +(38) Scan parquet default.customer_address Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(40) ColumnarToRow [codegen id : 14] +(39) ColumnarToRow [codegen id : 14] Input [2]: [ca_address_sk#22, ca_state#23] -(41) Filter [codegen id : 14] +(40) Filter [codegen id : 14] Input [2]: [ca_address_sk#22, ca_state#23] Condition : isnotnull(ca_address_sk#22) -(42) Exchange +(41) Exchange Input [2]: [ca_address_sk#22, ca_state#23] Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] -(43) Sort [codegen id : 15] +(42) Sort [codegen id : 15] Input [2]: [ca_address_sk#22, ca_state#23] Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 -(44) SortMergeJoin [codegen id : 16] +(43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#22] Join condition: None -(45) Project [codegen id : 16] +(44) Project [codegen id : 16] Output [2]: [c_current_cdemo_sk#4, ca_state#23] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] -(46) Exchange +(45) Exchange Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] -(47) Sort [codegen id : 17] +(46) Sort [codegen id : 17] Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(48) Scan parquet default.customer_demographics +(47) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 18] +(48) ColumnarToRow [codegen id : 18] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(50) Filter [codegen id : 18] +(49) Filter [codegen id : 18] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Condition : isnotnull(cd_demo_sk#26) -(51) Exchange +(50) Exchange Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] -(52) Sort [codegen id : 19] +(51) Sort [codegen id : 19] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 -(53) SortMergeJoin [codegen id : 20] +(52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#26] Join condition: None -(54) Project [codegen id : 20] +(53) Project [codegen id : 20] Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(55) HashAggregate [codegen id : 20] +(54) HashAggregate [codegen id : 20] Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#29), partial_max(cd_dep_count#29), partial_sum(cd_dep_count#29), partial_avg(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_sum(cd_dep_employed_count#30), partial_avg(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_sum(cd_dep_college_count#31)] Aggregate Attributes [13]: [count#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45] Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] -(56) Exchange +(55) Exchange Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] -(57) HashAggregate [codegen id : 21] +(56) HashAggregate [codegen id : 21] Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [10]: [count(1), avg(cd_dep_count#29), max(cd_dep_count#29), sum(cd_dep_count#29), avg(cd_dep_employed_count#30), max(cd_dep_employed_count#30), sum(cd_dep_employed_count#30), avg(cd_dep_college_count#31), max(cd_dep_college_count#31), sum(cd_dep_college_count#31)] Aggregate Attributes [10]: [count(1)#60, avg(cd_dep_count#29)#61, max(cd_dep_count#29)#62, sum(cd_dep_count#29)#63, avg(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, sum(cd_dep_employed_count#30)#66, avg(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, sum(cd_dep_college_count#31)#69] Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, count(1)#60 AS cnt1#70, avg(cd_dep_count#29)#61 AS avg(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, sum(cd_dep_count#29)#63 AS sum(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, avg(cd_dep_employed_count#30)#64 AS avg(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, sum(cd_dep_employed_count#30)#66 AS sum(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, avg(cd_dep_college_count#31)#67 AS avg(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, sum(cd_dep_college_count#31)#69 AS sum(cd_dep_college_count)#81] -(58) TakeOrderedAndProject +(57) TakeOrderedAndProject Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (59) +ReusedExchange (58) -(59) ReusedExchange [Reuses operator id: 13] +(58) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt index 7be34612be79f..ee441a8332a2a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -43,12 +43,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #6 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #6 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index c803c1989723d..12c8230c7e3bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -1,58 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (31) - : : +- * Filter (30) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (29) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (22) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Project (30) + : : +- * Filter (29) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (28) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (21) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (14) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (14) - : : : : +- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (21) - : : : +- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (18) - : : +- BroadcastExchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * ColumnarToRow (24) - : : : +- Scan parquet default.catalog_sales (23) - : : +- ReusedExchange (25) - : +- BroadcastExchange (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.customer_address (32) - +- BroadcastExchange (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.customer_demographics (38) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * ColumnarToRow (5) + : : : : : +- Scan parquet default.store_sales (4) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.date_dim (6) + : : : +- BroadcastExchange (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- BroadcastExchange (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (24) + : +- BroadcastExchange (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.customer_address (31) + +- BroadcastExchange (40) + +- * Filter (39) + +- * ColumnarToRow (38) + +- Scan parquet default.customer_demographics (37) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -60,220 +59,215 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) Filter [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_customer_sk#6) - -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -(11) BroadcastExchange +(10) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] -(12) BroadcastHashJoin [codegen id : 2] +(11) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join condition: None -(13) Project [codegen id : 2] +(12) Project [codegen id : 2] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(14) BroadcastExchange +(13) BroadcastExchange Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(15) BroadcastHashJoin [codegen id : 9] +(14) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join condition: None -(16) Scan parquet default.web_sales +(15) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#15), dynamicpruningexpression(ws_sold_date_sk#15 IN dynamicpruning#8)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] +(16) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] -(18) ReusedExchange [Reuses operator id: 11] +(17) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -(19) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] Right keys [1]: [d_date_sk#9] Join condition: None -(20) Project [codegen id : 4] +(19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] -(21) BroadcastExchange +(20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(22) BroadcastHashJoin [codegen id : 9] +(21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#14] Join condition: None -(23) Scan parquet default.catalog_sales +(22) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 6] +(23) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] -(25) ReusedExchange [Reuses operator id: 11] +(24) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -(26) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#9] Join condition: None -(27) Project [codegen id : 6] +(26) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#17] Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] -(28) BroadcastExchange +(27) BroadcastExchange Input [1]: [cs_ship_customer_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(29) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#17] Join condition: None -(30) Filter [codegen id : 9] +(29) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(31) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(32) Scan parquet default.customer_address +(31) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 7] +(32) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#20, ca_state#21] -(34) Filter [codegen id : 7] +(33) Filter [codegen id : 7] Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(35) BroadcastExchange +(34) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(36) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join condition: None -(37) Project [codegen id : 9] +(36) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, ca_state#21] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] -(38) Scan parquet default.customer_demographics +(37) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 8] +(38) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(40) Filter [codegen id : 8] +(39) Filter [codegen id : 8] Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Condition : isnotnull(cd_demo_sk#23) -(41) BroadcastExchange +(40) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(42) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join condition: None -(43) Project [codegen id : 9] +(42) Project [codegen id : 9] Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(44) HashAggregate [codegen id : 9] +(43) HashAggregate [codegen id : 9] Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] Aggregate Attributes [13]: [count#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42] Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] -(45) Exchange +(44) Exchange Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] -(46) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 10] Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] -(47) TakeOrderedAndProject +(46) TakeOrderedAndProject Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -ReusedExchange (48) +ReusedExchange (47) -(48) ReusedExchange [Reuses operator id: 11] +(47) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index c2dd037ca1b47..c6c9e1da58c26 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -23,12 +23,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt index bae142df5e6d6..1c1b8ad01abfa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt @@ -1,68 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (57) -+- * HashAggregate (56) - +- Exchange (55) - +- * HashAggregate (54) - +- * Project (53) - +- * SortMergeJoin Inner (52) - :- * Sort (46) - : +- Exchange (45) - : +- * Project (44) - : +- * SortMergeJoin Inner (43) - : :- * Sort (37) - : : +- Exchange (36) - : : +- * Project (35) - : : +- SortMergeJoin LeftSemi (34) - : : :- SortMergeJoin LeftSemi (18) +TakeOrderedAndProject (54) ++- * HashAggregate (53) + +- Exchange (52) + +- * HashAggregate (51) + +- * Project (50) + +- * SortMergeJoin Inner (49) + :- * Sort (43) + : +- Exchange (42) + : +- * Project (41) + : +- * SortMergeJoin Inner (40) + : :- * Sort (34) + : : +- Exchange (33) + : : +- * Project (32) + : : +- SortMergeJoin LeftSemi (31) + : : :- SortMergeJoin LeftSemi (17) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.customer (1) - : : : +- * Sort (17) - : : : +- Exchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * Filter (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.store_sales (6) - : : : +- BroadcastExchange (13) - : : : +- * Project (12) - : : : +- * Filter (11) - : : : +- * ColumnarToRow (10) - : : : +- Scan parquet default.date_dim (9) - : : +- * Sort (33) - : : +- Exchange (32) - : : +- Union (31) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (21) - : : : : +- * ColumnarToRow (20) - : : : : +- Scan parquet default.web_sales (19) - : : : +- ReusedExchange (22) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet default.catalog_sales (25) - : : +- ReusedExchange (28) - : +- * Sort (42) - : +- Exchange (41) - : +- * Filter (40) - : +- * ColumnarToRow (39) - : +- Scan parquet default.customer_address (38) - +- * Sort (51) - +- Exchange (50) - +- * Filter (49) - +- * ColumnarToRow (48) - +- Scan parquet default.customer_demographics (47) + : : : +- * Sort (16) + : : : +- Exchange (15) + : : : +- * Project (14) + : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : :- * ColumnarToRow (7) + : : : : +- Scan parquet default.store_sales (6) + : : : +- BroadcastExchange (12) + : : : +- * Project (11) + : : : +- * Filter (10) + : : : +- * ColumnarToRow (9) + : : : +- Scan parquet default.date_dim (8) + : : +- * Sort (30) + : : +- Exchange (29) + : : +- Union (28) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- Scan parquet default.web_sales (18) + : : : +- ReusedExchange (20) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * ColumnarToRow (24) + : : : +- Scan parquet default.catalog_sales (23) + : : +- ReusedExchange (25) + : +- * Sort (39) + : +- Exchange (38) + : +- * Filter (37) + : +- * ColumnarToRow (36) + : +- Scan parquet default.customer_address (35) + +- * Sort (48) + +- Exchange (47) + +- * Filter (46) + +- * ColumnarToRow (45) + +- Scan parquet default.customer_demographics (44) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -70,7 +67,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -85,244 +82,229 @@ Output [2]: [ss_customer_sk#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -(8) Filter [codegen id : 4] -Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_customer_sk#5) - -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] Condition : ((((isnotnull(d_year#9) AND isnotnull(d_qoy#10)) AND (d_year#9 = 1999)) AND (d_qoy#10 < 4)) AND isnotnull(d_date_sk#8)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [1]: [d_date_sk#8] Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -(13) BroadcastExchange +(12) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#8] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [1]: [ss_customer_sk#5] Input [3]: [ss_customer_sk#5, ss_sold_date_sk#6, d_date_sk#8] -(16) Exchange +(15) Exchange Input [1]: [ss_customer_sk#5] Arguments: hashpartitioning(ss_customer_sk#5, 5), ENSURE_REQUIREMENTS, [id=#12] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [1]: [ss_customer_sk#5] Arguments: [ss_customer_sk#5 ASC NULLS FIRST], false, 0 -(18) SortMergeJoin +(17) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#5] Join condition: None -(19) Scan parquet default.web_sales +(18) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#14), dynamicpruningexpression(ws_sold_date_sk#14 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 7] -Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] - -(21) Filter [codegen id : 7] +(19) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] -Condition : isnotnull(ws_bill_customer_sk#13) -(22) ReusedExchange [Reuses operator id: 13] +(20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -(23) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] Right keys [1]: [d_date_sk#8] Join condition: None -(24) Project [codegen id : 7] +(22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13 AS customsk#15] Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] -(25) Scan parquet default.catalog_sales +(23) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 9] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] - -(27) Filter [codegen id : 9] +(24) ColumnarToRow [codegen id : 9] Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Condition : isnotnull(cs_ship_customer_sk#16) -(28) ReusedExchange [Reuses operator id: 13] +(25) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -(29) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#8] Join condition: None -(30) Project [codegen id : 9] +(27) Project [codegen id : 9] Output [1]: [cs_ship_customer_sk#16 AS customsk#18] Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] -(31) Union +(28) Union -(32) Exchange +(29) Exchange Input [1]: [customsk#15] Arguments: hashpartitioning(customsk#15, 5), ENSURE_REQUIREMENTS, [id=#19] -(33) Sort [codegen id : 10] +(30) Sort [codegen id : 10] Input [1]: [customsk#15] Arguments: [customsk#15 ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(31) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [customsk#15] Join condition: None -(35) Project [codegen id : 11] +(32) Project [codegen id : 11] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(36) Exchange +(33) Exchange Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#20] -(37) Sort [codegen id : 12] +(34) Sort [codegen id : 12] Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: [c_current_addr_sk#3 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.customer_address +(35) Scan parquet default.customer_address Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 13] +(36) ColumnarToRow [codegen id : 13] Input [2]: [ca_address_sk#21, ca_state#22] -(40) Filter [codegen id : 13] +(37) Filter [codegen id : 13] Input [2]: [ca_address_sk#21, ca_state#22] Condition : isnotnull(ca_address_sk#21) -(41) Exchange +(38) Exchange Input [2]: [ca_address_sk#21, ca_state#22] Arguments: hashpartitioning(ca_address_sk#21, 5), ENSURE_REQUIREMENTS, [id=#23] -(42) Sort [codegen id : 14] +(39) Sort [codegen id : 14] Input [2]: [ca_address_sk#21, ca_state#22] Arguments: [ca_address_sk#21 ASC NULLS FIRST], false, 0 -(43) SortMergeJoin [codegen id : 15] +(40) SortMergeJoin [codegen id : 15] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#21] Join condition: None -(44) Project [codegen id : 15] +(41) Project [codegen id : 15] Output [2]: [c_current_cdemo_sk#2, ca_state#22] Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21, ca_state#22] -(45) Exchange +(42) Exchange Input [2]: [c_current_cdemo_sk#2, ca_state#22] Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] -(46) Sort [codegen id : 16] +(43) Sort [codegen id : 16] Input [2]: [c_current_cdemo_sk#2, ca_state#22] Arguments: [c_current_cdemo_sk#2 ASC NULLS FIRST], false, 0 -(47) Scan parquet default.customer_demographics +(44) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 17] +(45) ColumnarToRow [codegen id : 17] Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(49) Filter [codegen id : 17] +(46) Filter [codegen id : 17] Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Condition : isnotnull(cd_demo_sk#25) -(50) Exchange +(47) Exchange Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] -(51) Sort [codegen id : 18] +(48) Sort [codegen id : 18] Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 -(52) SortMergeJoin [codegen id : 19] +(49) SortMergeJoin [codegen id : 19] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#25] Join condition: None -(53) Project [codegen id : 19] +(50) Project [codegen id : 19] Output [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(54) HashAggregate [codegen id : 19] +(51) HashAggregate [codegen id : 19] Input [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#28), partial_max(cd_dep_count#28), partial_sum(cd_dep_count#28), partial_avg(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_sum(cd_dep_employed_count#29), partial_avg(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_sum(cd_dep_college_count#30)] Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] Results [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -(55) Exchange +(52) Exchange Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] Arguments: hashpartitioning(ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] -(56) HashAggregate [codegen id : 20] +(53) HashAggregate [codegen id : 20] Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [10]: [count(1), avg(cd_dep_count#28), max(cd_dep_count#28), sum(cd_dep_count#28), avg(cd_dep_employed_count#29), max(cd_dep_employed_count#29), sum(cd_dep_employed_count#29), avg(cd_dep_college_count#30), max(cd_dep_college_count#30), sum(cd_dep_college_count#30)] Aggregate Attributes [10]: [count(1)#59, avg(cd_dep_count#28)#60, max(cd_dep_count#28)#61, sum(cd_dep_count#28)#62, avg(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, sum(cd_dep_employed_count#29)#65, avg(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, sum(cd_dep_college_count#30)#68] Results [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, count(1)#59 AS cnt1#69, avg(cd_dep_count#28)#60 AS avg(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, sum(cd_dep_count#28)#62 AS sum(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, avg(cd_dep_employed_count#29)#63 AS avg(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, sum(cd_dep_employed_count#29)#65 AS sum(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, avg(cd_dep_college_count#30)#66 AS avg(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, sum(cd_dep_college_count#30)#68 AS sum(cd_dep_college_count)#80] -(57) TakeOrderedAndProject +(54) TakeOrderedAndProject Input [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -ReusedExchange (58) +ReusedExchange (55) -(58) ReusedExchange [Reuses operator id: 13] +(55) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt index 419706064ea82..9a090e0da62c8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt @@ -30,7 +30,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -41,12 +41,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #6 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #6 WholeStageCodegen (3) @@ -63,21 +62,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (7) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #6 WholeStageCodegen (9) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index 12f35bdc6bb56..ffb07d398de71 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -1,58 +1,55 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (31) - : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : :- * BroadcastHashJoin LeftSemi BuildRight (15) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (28) + : : +- * BroadcastHashJoin LeftSemi BuildRight (27) + : : :- * BroadcastHashJoin LeftSemi BuildRight (14) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.customer (1) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.store_sales (4) - : : : +- BroadcastExchange (11) - : : : +- * Project (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.date_dim (7) - : : +- BroadcastExchange (29) - : : +- Union (28) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (19) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.customer_address (32) - +- BroadcastExchange (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.customer_demographics (38) + : : : +- BroadcastExchange (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * ColumnarToRow (5) + : : : : +- Scan parquet default.store_sales (4) + : : : +- BroadcastExchange (10) + : : : +- * Project (9) + : : : +- * Filter (8) + : : : +- * ColumnarToRow (7) + : : : +- Scan parquet default.date_dim (6) + : : +- BroadcastExchange (26) + : : +- Union (25) + : : :- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * ColumnarToRow (21) + : : : +- Scan parquet default.catalog_sales (20) + : : +- ReusedExchange (22) + : +- BroadcastExchange (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet default.customer_address (29) + +- BroadcastExchange (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.customer_demographics (35) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -60,219 +57,204 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) Filter [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Condition : isnotnull(ss_customer_sk#4) - -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [1]: [d_date_sk#7] Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -(11) BroadcastExchange +(10) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] -(12) BroadcastHashJoin [codegen id : 2] +(11) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join condition: None -(13) Project [codegen id : 2] +(12) Project [codegen id : 2] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -(14) BroadcastExchange +(13) BroadcastExchange Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(15) BroadcastHashJoin [codegen id : 9] +(14) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join condition: None -(16) Scan parquet default.web_sales +(15) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] - -(18) Filter [codegen id : 4] +(16) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Condition : isnotnull(ws_bill_customer_sk#12) -(19) ReusedExchange [Reuses operator id: 11] +(17) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -(20) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] Right keys [1]: [d_date_sk#7] Join condition: None -(21) Project [codegen id : 4] +(19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12 AS customsk#14] Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] -(22) Scan parquet default.catalog_sales +(20) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] - -(24) Filter [codegen id : 6] +(21) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_ship_customer_sk#15) -(25) ReusedExchange [Reuses operator id: 11] +(22) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -(26) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#7] Join condition: None -(27) Project [codegen id : 6] +(24) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#15 AS customsk#17] Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] -(28) Union +(25) Union -(29) BroadcastExchange +(26) BroadcastExchange Input [1]: [customsk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(30) BroadcastHashJoin [codegen id : 9] +(27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customsk#14] Join condition: None -(31) Project [codegen id : 9] +(28) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(32) Scan parquet default.customer_address +(29) Scan parquet default.customer_address Output [2]: [ca_address_sk#19, ca_state#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 7] +(30) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#19, ca_state#20] -(34) Filter [codegen id : 7] +(31) Filter [codegen id : 7] Input [2]: [ca_address_sk#19, ca_state#20] Condition : isnotnull(ca_address_sk#19) -(35) BroadcastExchange +(32) BroadcastExchange Input [2]: [ca_address_sk#19, ca_state#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] -(36) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#19] Join condition: None -(37) Project [codegen id : 9] +(34) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, ca_state#20] Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19, ca_state#20] -(38) Scan parquet default.customer_demographics +(35) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 8] +(36) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(40) Filter [codegen id : 8] +(37) Filter [codegen id : 8] Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Condition : isnotnull(cd_demo_sk#22) -(41) BroadcastExchange +(38) BroadcastExchange Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] -(42) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#22] Join condition: None -(43) Project [codegen id : 9] +(40) Project [codegen id : 9] Output [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Input [8]: [c_current_cdemo_sk#2, ca_state#20, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(44) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 9] Input [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] Results [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -(45) Exchange +(42) Exchange Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] Arguments: hashpartitioning(ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [id=#55] -(46) HashAggregate [codegen id : 10] +(43) HashAggregate [codegen id : 10] Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] Aggregate Attributes [10]: [count(1)#56, avg(cd_dep_count#25)#57, max(cd_dep_count#25)#58, sum(cd_dep_count#25)#59, avg(cd_dep_employed_count#26)#60, max(cd_dep_employed_count#26)#61, sum(cd_dep_employed_count#26)#62, avg(cd_dep_college_count#27)#63, max(cd_dep_college_count#27)#64, sum(cd_dep_college_count#27)#65] Results [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#56 AS cnt1#66, avg(cd_dep_count#25)#57 AS avg(cd_dep_count)#67, max(cd_dep_count#25)#58 AS max(cd_dep_count)#68, sum(cd_dep_count#25)#59 AS sum(cd_dep_count)#69, cd_dep_employed_count#26, count(1)#56 AS cnt2#70, avg(cd_dep_employed_count#26)#60 AS avg(cd_dep_employed_count)#71, max(cd_dep_employed_count#26)#61 AS max(cd_dep_employed_count)#72, sum(cd_dep_employed_count#26)#62 AS sum(cd_dep_employed_count)#73, cd_dep_college_count#27, count(1)#56 AS cnt3#74, avg(cd_dep_college_count#27)#63 AS avg(cd_dep_college_count)#75, max(cd_dep_college_count#27)#64 AS max(cd_dep_college_count)#76, sum(cd_dep_college_count#27)#65 AS sum(cd_dep_college_count)#77] -(47) TakeOrderedAndProject +(44) TakeOrderedAndProject Input [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (48) +ReusedExchange (45) -(48) ReusedExchange [Reuses operator id: 11] +(45) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index ae82b13a9533e..1ab7d548e59dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customsk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -21,12 +21,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,21 +40,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt index 849d2f418a9a8..5374a708a0295 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (95) -+- * HashAggregate (94) - +- Exchange (93) - +- * HashAggregate (92) - +- Union (91) - :- * HashAggregate (80) - : +- Exchange (79) - : +- * HashAggregate (78) - : +- Union (77) +TakeOrderedAndProject (94) ++- * HashAggregate (93) + +- Exchange (92) + +- * HashAggregate (91) + +- Union (90) + :- * HashAggregate (79) + : +- Exchange (78) + : +- * HashAggregate (77) + : +- Union (76) : :- * HashAggregate (25) : : +- Exchange (24) : : +- * HashAggregate (23) @@ -54,46 +54,45 @@ TakeOrderedAndProject (95) : : : +- * ColumnarToRow (36) : : : +- Scan parquet default.catalog_page (35) : : +- ReusedExchange (41) - : +- * HashAggregate (76) - : +- Exchange (75) - : +- * HashAggregate (74) - : +- * Project (73) - : +- * BroadcastHashJoin Inner BuildRight (72) - : :- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- Union (64) + : +- * HashAggregate (75) + : +- Exchange (74) + : +- * HashAggregate (73) + : +- * Project (72) + : +- * BroadcastHashJoin Inner BuildRight (71) + : :- * Project (69) + : : +- * BroadcastHashJoin Inner BuildRight (68) + : : :- Union (63) : : : :- * Project (50) : : : : +- * Filter (49) : : : : +- * ColumnarToRow (48) : : : : +- Scan parquet default.web_sales (47) - : : : +- * Project (63) - : : : +- * SortMergeJoin Inner (62) - : : : :- * Sort (55) - : : : : +- Exchange (54) - : : : : +- * Filter (53) - : : : : +- * ColumnarToRow (52) - : : : : +- Scan parquet default.web_returns (51) - : : : +- * Sort (61) - : : : +- Exchange (60) - : : : +- * Project (59) - : : : +- * Filter (58) - : : : +- * ColumnarToRow (57) - : : : +- Scan parquet default.web_sales (56) - : : +- BroadcastExchange (68) - : : +- * Filter (67) - : : +- * ColumnarToRow (66) - : : +- Scan parquet default.web_site (65) - : +- ReusedExchange (71) - :- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * HashAggregate (82) - : +- ReusedExchange (81) - +- * HashAggregate (90) - +- Exchange (89) - +- * HashAggregate (88) - +- * HashAggregate (87) - +- ReusedExchange (86) + : : : +- * Project (62) + : : : +- * SortMergeJoin Inner (61) + : : : :- * Sort (54) + : : : : +- Exchange (53) + : : : : +- * ColumnarToRow (52) + : : : : +- Scan parquet default.web_returns (51) + : : : +- * Sort (60) + : : : +- Exchange (59) + : : : +- * Project (58) + : : : +- * Filter (57) + : : : +- * ColumnarToRow (56) + : : : +- Scan parquet default.web_sales (55) + : : +- BroadcastExchange (67) + : : +- * Filter (66) + : : +- * ColumnarToRow (65) + : : +- Scan parquet default.web_site (64) + : +- ReusedExchange (70) + :- * HashAggregate (84) + : +- Exchange (83) + : +- * HashAggregate (82) + : +- * HashAggregate (81) + : +- ReusedExchange (80) + +- * HashAggregate (89) + +- Exchange (88) + +- * HashAggregate (87) + +- * HashAggregate (86) + +- ReusedExchange (85) (1) Scan parquet default.store_sales @@ -332,234 +331,229 @@ Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#1 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -(53) Filter [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Condition : (isnotnull(wr_item_sk#98) AND isnotnull(wr_order_number#99)) - -(54) Exchange +(53) Exchange Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] -(55) Sort [codegen id : 15] +(54) Sort [codegen id : 15] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 -(56) Scan parquet default.web_sales +(55) Scan parquet default.web_sales Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(57) ColumnarToRow [codegen id : 16] +(56) ColumnarToRow [codegen id : 16] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(58) Filter [codegen id : 16] +(57) Filter [codegen id : 16] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) -(59) Project [codegen id : 16] +(58) Project [codegen id : 16] Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(60) Exchange +(59) Exchange Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] -(61) Sort [codegen id : 17] +(60) Sort [codegen id : 17] Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 -(62) SortMergeJoin [codegen id : 18] +(61) SortMergeJoin [codegen id : 18] Left keys [2]: [wr_item_sk#98, wr_order_number#99] Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None -(63) Project [codegen id : 18] +(62) Project [codegen id : 18] Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -(64) Union +(63) Union -(65) Scan parquet default.web_site +(64) Scan parquet default.web_site Output [2]: [web_site_sk#113, web_site_id#114] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] +(65) ColumnarToRow [codegen id : 19] Input [2]: [web_site_sk#113, web_site_id#114] -(67) Filter [codegen id : 19] +(66) Filter [codegen id : 19] Input [2]: [web_site_sk#113, web_site_id#114] Condition : isnotnull(web_site_sk#113) -(68) BroadcastExchange +(67) BroadcastExchange Input [2]: [web_site_sk#113, web_site_id#114] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] -(69) BroadcastHashJoin [codegen id : 21] +(68) BroadcastHashJoin [codegen id : 21] Left keys [1]: [wsr_web_site_sk#92] Right keys [1]: [web_site_sk#113] Join condition: None -(70) Project [codegen id : 21] +(69) Project [codegen id : 21] Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] -(71) ReusedExchange [Reuses operator id: 20] +(70) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] -(72) BroadcastHashJoin [codegen id : 21] +(71) BroadcastHashJoin [codegen id : 21] Left keys [1]: [date_sk#93] Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None -(73) Project [codegen id : 21] +(72) Project [codegen id : 21] Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] -(74) HashAggregate [codegen id : 21] +(73) HashAggregate [codegen id : 21] Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Keys [1]: [web_site_id#114] Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -(75) Exchange +(74) Exchange Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] -(76) HashAggregate [codegen id : 22] +(75) HashAggregate [codegen id : 22] Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] Keys [1]: [web_site_id#114] Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] Results [5]: [web channel AS channel#129, concat(web_site, web_site_id#114) AS id#130, MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#131, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS returns#132, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#133] -(77) Union +(76) Union -(78) HashAggregate [codegen id : 23] +(77) HashAggregate [codegen id : 23] Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] Aggregate Attributes [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] Results [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -(79) Exchange +(78) Exchange Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#146] -(80) HashAggregate [codegen id : 24] +(79) HashAggregate [codegen id : 24] Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] Aggregate Attributes [3]: [sum(sales#43)#147, sum(returns#44)#148, sum(profit#45)#149] Results [5]: [channel#41, id#42, cast(sum(sales#43)#147 as decimal(37,2)) AS sales#150, cast(sum(returns#44)#148 as decimal(37,2)) AS returns#151, cast(sum(profit#45)#149 as decimal(38,2)) AS profit#152] -(81) ReusedExchange [Reuses operator id: unknown] +(80) ReusedExchange [Reuses operator id: unknown] Output [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] -(82) HashAggregate [codegen id : 48] +(81) HashAggregate [codegen id : 48] Input [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#159)] Aggregate Attributes [3]: [sum(sales#43)#160, sum(returns#44)#161, sum(profit#159)#162] Results [4]: [channel#41, sum(sales#43)#160 AS sales#163, sum(returns#44)#161 AS returns#164, sum(profit#159)#162 AS profit#165] -(83) HashAggregate [codegen id : 48] +(82) HashAggregate [codegen id : 48] Input [4]: [channel#41, sales#163, returns#164, profit#165] Keys [1]: [channel#41] Functions [3]: [partial_sum(sales#163), partial_sum(returns#164), partial_sum(profit#165)] Aggregate Attributes [6]: [sum#166, isEmpty#167, sum#168, isEmpty#169, sum#170, isEmpty#171] Results [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] -(84) Exchange +(83) Exchange Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#178] -(85) HashAggregate [codegen id : 49] +(84) HashAggregate [codegen id : 49] Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] Keys [1]: [channel#41] Functions [3]: [sum(sales#163), sum(returns#164), sum(profit#165)] Aggregate Attributes [3]: [sum(sales#163)#179, sum(returns#164)#180, sum(profit#165)#181] Results [5]: [channel#41, null AS id#182, sum(sales#163)#179 AS sum(sales)#183, sum(returns#164)#180 AS sum(returns)#184, sum(profit#165)#181 AS sum(profit)#185] -(86) ReusedExchange [Reuses operator id: unknown] +(85) ReusedExchange [Reuses operator id: unknown] Output [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] -(87) HashAggregate [codegen id : 73] +(86) HashAggregate [codegen id : 73] Input [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#192)] Aggregate Attributes [3]: [sum(sales#43)#193, sum(returns#44)#194, sum(profit#192)#195] Results [3]: [sum(sales#43)#193 AS sales#163, sum(returns#44)#194 AS returns#164, sum(profit#192)#195 AS profit#165] -(88) HashAggregate [codegen id : 73] +(87) HashAggregate [codegen id : 73] Input [3]: [sales#163, returns#164, profit#165] Keys: [] Functions [3]: [partial_sum(sales#163), partial_sum(returns#164), partial_sum(profit#165)] Aggregate Attributes [6]: [sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201] Results [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] -(89) Exchange +(88) Exchange Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#208] -(90) HashAggregate [codegen id : 74] +(89) HashAggregate [codegen id : 74] Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] Keys: [] Functions [3]: [sum(sales#163), sum(returns#164), sum(profit#165)] Aggregate Attributes [3]: [sum(sales#163)#209, sum(returns#164)#210, sum(profit#165)#211] Results [5]: [null AS channel#212, null AS id#213, sum(sales#163)#209 AS sum(sales)#214, sum(returns#164)#210 AS sum(returns)#215, sum(profit#165)#211 AS sum(profit)#216] -(91) Union +(90) Union -(92) HashAggregate [codegen id : 75] +(91) HashAggregate [codegen id : 75] Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Functions: [] Aggregate Attributes: [] Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -(93) Exchange +(92) Exchange Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Arguments: hashpartitioning(channel#41, id#42, sales#150, returns#151, profit#152, 5), ENSURE_REQUIREMENTS, [id=#217] -(94) HashAggregate [codegen id : 76] +(93) HashAggregate [codegen id : 76] Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Functions: [] Aggregate Attributes: [] Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -(95) TakeOrderedAndProject +(94) TakeOrderedAndProject Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#150, returns#151, profit#152] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5 -ReusedExchange (96) +ReusedExchange (95) -(96) ReusedExchange [Reuses operator id: 20] +(95) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#49 IN dynamicpruning#50 -ReusedExchange (97) +ReusedExchange (96) -(97) ReusedExchange [Reuses operator id: 20] +(96) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt index 9a3d775ea8266..dd2c25b96568b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt @@ -121,11 +121,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Exchange [wr_item_sk,wr_order_number] #9 WholeStageCodegen (14) - Filter [wr_item_sk,wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter WholeStageCodegen (17) Sort [ws_item_sk,ws_order_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index a5e7d0339212f..89362fb4f0efe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (92) -+- * HashAggregate (91) - +- Exchange (90) - +- * HashAggregate (89) - +- Union (88) - :- * HashAggregate (77) - : +- Exchange (76) - : +- * HashAggregate (75) - : +- Union (74) +TakeOrderedAndProject (91) ++- * HashAggregate (90) + +- Exchange (89) + +- * HashAggregate (88) + +- Union (87) + :- * HashAggregate (76) + : +- Exchange (75) + : +- * HashAggregate (74) + : +- Union (73) : :- * HashAggregate (25) : : +- Exchange (24) : : +- * HashAggregate (23) @@ -54,43 +54,42 @@ TakeOrderedAndProject (92) : : +- * Filter (40) : : +- * ColumnarToRow (39) : : +- Scan parquet default.catalog_page (38) - : +- * HashAggregate (73) - : +- Exchange (72) - : +- * HashAggregate (71) - : +- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- Union (61) + : +- * HashAggregate (72) + : +- Exchange (71) + : +- * HashAggregate (70) + : +- * Project (69) + : +- * BroadcastHashJoin Inner BuildRight (68) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- Union (60) : : : :- * Project (50) : : : : +- * Filter (49) : : : : +- * ColumnarToRow (48) : : : : +- Scan parquet default.web_sales (47) - : : : +- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildLeft (59) - : : : :- BroadcastExchange (54) - : : : : +- * Filter (53) - : : : : +- * ColumnarToRow (52) - : : : : +- Scan parquet default.web_returns (51) - : : : +- * Project (58) - : : : +- * Filter (57) - : : : +- * ColumnarToRow (56) - : : : +- Scan parquet default.web_sales (55) - : : +- ReusedExchange (62) - : +- BroadcastExchange (68) - : +- * Filter (67) - : +- * ColumnarToRow (66) - : +- Scan parquet default.web_site (65) - :- * HashAggregate (82) - : +- Exchange (81) - : +- * HashAggregate (80) - : +- * HashAggregate (79) - : +- ReusedExchange (78) - +- * HashAggregate (87) - +- Exchange (86) - +- * HashAggregate (85) - +- * HashAggregate (84) - +- ReusedExchange (83) + : : : +- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildLeft (58) + : : : :- BroadcastExchange (53) + : : : : +- * ColumnarToRow (52) + : : : : +- Scan parquet default.web_returns (51) + : : : +- * Project (57) + : : : +- * Filter (56) + : : : +- * ColumnarToRow (55) + : : : +- Scan parquet default.web_sales (54) + : : +- ReusedExchange (61) + : +- BroadcastExchange (67) + : +- * Filter (66) + : +- * ColumnarToRow (65) + : +- Scan parquet default.web_site (64) + :- * HashAggregate (81) + : +- Exchange (80) + : +- * HashAggregate (79) + : +- * HashAggregate (78) + : +- ReusedExchange (77) + +- * HashAggregate (86) + +- Exchange (85) + +- * HashAggregate (84) + +- * HashAggregate (83) + +- ReusedExchange (82) (1) Scan parquet default.store_sales @@ -329,222 +328,217 @@ Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#1 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -(53) Filter [codegen id : 14] +(53) BroadcastExchange Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Condition : (isnotnull(wr_item_sk#98) AND isnotnull(wr_order_number#99)) +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] -(54) BroadcastExchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false], input[1, bigint, false]),false), [id=#103] - -(55) Scan parquet default.web_sales +(54) Scan parquet default.web_sales Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(56) ColumnarToRow +(55) ColumnarToRow Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(57) Filter +(56) Filter Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) -(58) Project +(57) Project Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(59) BroadcastHashJoin [codegen id : 15] +(58) BroadcastHashJoin [codegen id : 15] Left keys [2]: [wr_item_sk#98, wr_order_number#99] Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None -(60) Project [codegen id : 15] +(59) Project [codegen id : 15] Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -(61) Union +(60) Union -(62) ReusedExchange [Reuses operator id: 14] +(61) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] -(63) BroadcastHashJoin [codegen id : 18] +(62) BroadcastHashJoin [codegen id : 18] Left keys [1]: [date_sk#93] Right keys [1]: [cast(d_date_sk#22 as bigint)] Join condition: None -(64) Project [codegen id : 18] +(63) Project [codegen id : 18] Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] -(65) Scan parquet default.web_site +(64) Scan parquet default.web_site Output [2]: [web_site_sk#112, web_site_id#113] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 17] +(65) ColumnarToRow [codegen id : 17] Input [2]: [web_site_sk#112, web_site_id#113] -(67) Filter [codegen id : 17] +(66) Filter [codegen id : 17] Input [2]: [web_site_sk#112, web_site_id#113] Condition : isnotnull(web_site_sk#112) -(68) BroadcastExchange +(67) BroadcastExchange Input [2]: [web_site_sk#112, web_site_id#113] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] -(69) BroadcastHashJoin [codegen id : 18] +(68) BroadcastHashJoin [codegen id : 18] Left keys [1]: [wsr_web_site_sk#92] Right keys [1]: [web_site_sk#112] Join condition: None -(70) Project [codegen id : 18] +(69) Project [codegen id : 18] Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] -(71) HashAggregate [codegen id : 18] +(70) HashAggregate [codegen id : 18] Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] Keys [1]: [web_site_id#113] Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -(72) Exchange +(71) Exchange Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] -(73) HashAggregate [codegen id : 19] +(72) HashAggregate [codegen id : 19] Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] Keys [1]: [web_site_id#113] Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] Results [5]: [web channel AS channel#128, concat(web_site, web_site_id#113) AS id#129, MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#130, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS returns#131, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#132] -(74) Union +(73) Union -(75) HashAggregate [codegen id : 20] +(74) HashAggregate [codegen id : 20] Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] Results [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -(76) Exchange +(75) Exchange Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#145] -(77) HashAggregate [codegen id : 21] +(76) HashAggregate [codegen id : 21] Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] Aggregate Attributes [3]: [sum(sales#43)#146, sum(returns#44)#147, sum(profit#45)#148] Results [5]: [channel#41, id#42, cast(sum(sales#43)#146 as decimal(37,2)) AS sales#149, cast(sum(returns#44)#147 as decimal(37,2)) AS returns#150, cast(sum(profit#45)#148 as decimal(38,2)) AS profit#151] -(78) ReusedExchange [Reuses operator id: unknown] +(77) ReusedExchange [Reuses operator id: unknown] Output [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] -(79) HashAggregate [codegen id : 42] +(78) HashAggregate [codegen id : 42] Input [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#158)] Aggregate Attributes [3]: [sum(sales#43)#159, sum(returns#44)#160, sum(profit#158)#161] Results [4]: [channel#41, sum(sales#43)#159 AS sales#162, sum(returns#44)#160 AS returns#163, sum(profit#158)#161 AS profit#164] -(80) HashAggregate [codegen id : 42] +(79) HashAggregate [codegen id : 42] Input [4]: [channel#41, sales#162, returns#163, profit#164] Keys [1]: [channel#41] Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] Aggregate Attributes [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] Results [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -(81) Exchange +(80) Exchange Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#177] -(82) HashAggregate [codegen id : 43] +(81) HashAggregate [codegen id : 43] Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Keys [1]: [channel#41] Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] Aggregate Attributes [3]: [sum(sales#162)#178, sum(returns#163)#179, sum(profit#164)#180] Results [5]: [channel#41, null AS id#181, sum(sales#162)#178 AS sum(sales)#182, sum(returns#163)#179 AS sum(returns)#183, sum(profit#164)#180 AS sum(profit)#184] -(83) ReusedExchange [Reuses operator id: unknown] +(82) ReusedExchange [Reuses operator id: unknown] Output [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] -(84) HashAggregate [codegen id : 64] +(83) HashAggregate [codegen id : 64] Input [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#191)] Aggregate Attributes [3]: [sum(sales#43)#192, sum(returns#44)#193, sum(profit#191)#194] Results [3]: [sum(sales#43)#192 AS sales#162, sum(returns#44)#193 AS returns#163, sum(profit#191)#194 AS profit#164] -(85) HashAggregate [codegen id : 64] +(84) HashAggregate [codegen id : 64] Input [3]: [sales#162, returns#163, profit#164] Keys: [] Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] Aggregate Attributes [6]: [sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200] Results [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] -(86) Exchange +(85) Exchange Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#207] -(87) HashAggregate [codegen id : 65] +(86) HashAggregate [codegen id : 65] Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Keys: [] Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] Aggregate Attributes [3]: [sum(sales#162)#208, sum(returns#163)#209, sum(profit#164)#210] Results [5]: [null AS channel#211, null AS id#212, sum(sales#162)#208 AS sum(sales)#213, sum(returns#163)#209 AS sum(returns)#214, sum(profit#164)#210 AS sum(profit)#215] -(88) Union +(87) Union -(89) HashAggregate [codegen id : 66] +(88) HashAggregate [codegen id : 66] Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Functions: [] Aggregate Attributes: [] Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -(90) Exchange +(89) Exchange Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Arguments: hashpartitioning(channel#41, id#42, sales#149, returns#150, profit#151, 5), ENSURE_REQUIREMENTS, [id=#216] -(91) HashAggregate [codegen id : 67] +(90) HashAggregate [codegen id : 67] Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Functions: [] Aggregate Attributes: [] Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -(92) TakeOrderedAndProject +(91) TakeOrderedAndProject Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#149, returns#150, profit#151] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5 -ReusedExchange (93) +ReusedExchange (92) -(93) ReusedExchange [Reuses operator id: 14] +(92) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#49 IN dynamicpruning#50 -ReusedExchange (94) +ReusedExchange (93) -(94) ReusedExchange [Reuses operator id: 14] +(93) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index a787e89a028aa..caa7fcf4957ca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -118,11 +118,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #9 WholeStageCodegen (14) - Filter [wr_item_sk,wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 Project [ws_item_sk,ws_web_site_sk,ws_order_number] Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt index 09545881f17c3..3c65529504320 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt @@ -111,7 +111,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#8, s_county#9, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] +PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -119,7 +119,7 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_state#10)) +Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : ((isnotnull(ranking#19) AND (ranking#19 <= 5)) AND isnotnull(s_state#16)) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt index 00cad8c468868..c1b7cfef5638c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk,s_state] + Filter [s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_county,s_state] @@ -51,7 +51,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #6 WholeStageCodegen (7) Project [s_state] - Filter [ranking,s_state] + Filter [ranking] InputAdapter Window [_w2,s_state] WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index de78b4ff4cdcc..406acb0e0a27f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -111,7 +111,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#8, s_county#9, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] +PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -119,7 +119,7 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_state#10)) +Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : ((isnotnull(ranking#19) AND (ranking#19 <= 5)) AND isnotnull(s_state#16)) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index b6fdfb5868b9c..cd1e7e69e65d0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk,s_state] + Filter [s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_county,s_state] @@ -51,7 +51,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #6 WholeStageCodegen (7) Project [s_state] - Filter [ranking,s_state] + Filter [ranking] InputAdapter Window [_w2,s_state] WholeStageCodegen (6) From c63b162abc0e9b486f1422b5de67456435a99d2d Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sat, 13 Feb 2021 18:29:51 +0800 Subject: [PATCH 9/9] fix --- .../approved-plans-v1_4/q16.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q16/explain.txt | 2 +- .../approved-plans-v1_4/q33.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q33/explain.txt | 4 ++-- .../approved-plans-v1_4/q5.sf100/explain.txt | 2 +- .../tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt | 2 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q58/explain.txt | 2 +- .../approved-plans-v1_4/q93.sf100/explain.txt | 4 ++-- .../approved-plans-v1_4/q93/explain.txt | 4 ++-- .../approved-plans-v1_4/q94.sf100/explain.txt | 6 +++--- .../approved-plans-v1_4/q94/explain.txt | 6 +++--- .../approved-plans-v1_4/q95.sf100/explain.txt | 6 +++--- .../approved-plans-v1_4/q95/explain.txt | 6 +++--- .../approved-plans-v2_7/q5a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q5a/explain.txt | 2 +- 16 files changed, 28 insertions(+), 28 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index 9c56438870f64..adfa3e44f996d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -203,7 +203,7 @@ Input [2]: [d_date_sk#22, d_date#23] (36) Filter [codegen id : 10] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2002-02-01)) AND (d_date#23 <= 2002-04-02)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11719)) AND (d_date#23 <= 11779)) AND isnotnull(d_date_sk#22)) (37) Project [codegen id : 10] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index dde6a9f564859..5640564564396 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -141,7 +141,7 @@ Input [2]: [d_date_sk#16, d_date#17] (22) Filter [codegen id : 8] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2002-02-01)) AND (d_date#17 <= 2002-04-02)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 11719)) AND (d_date#17 <= 11779)) AND isnotnull(d_date_sk#16)) (23) Project [codegen id : 8] Output [1]: [d_date_sk#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index 1349aa54f3cc2..cba099c2bb5b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -163,7 +163,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index 1349aa54f3cc2..cba099c2bb5b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -163,7 +163,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index a754cef8f2962..2d3dea5d84565 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -160,7 +160,7 @@ Input [2]: [d_date_sk#25, d_date#26] (18) Filter [codegen id : 4] Input [2]: [d_date_sk#25, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 2000-08-23)) AND (d_date#26 <= 2000-09-06)) AND isnotnull(d_date_sk#25)) +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 11192)) AND (d_date#26 <= 11206)) AND isnotnull(d_date_sk#25)) (19) Project [codegen id : 4] Output [1]: [d_date_sk#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index b3eb1f9592ff3..f6fe8bee89369 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -130,7 +130,7 @@ Input [2]: [d_date_sk#22, d_date#23] (12) Filter [codegen id : 3] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11192)) AND (d_date#23 <= 11206)) AND isnotnull(d_date_sk#22)) (13) Project [codegen id : 3] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index f6dd5a59de4d4..ee3abb39ed053 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -467,7 +467,7 @@ Input [2]: [d_date#5, d_week_seq#6] (82) Filter [codegen id : 1] Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_date#5) AND (d_date#5 = 2000-01-03)) +Condition : (isnotnull(d_date#5) AND (d_date#5 = 10959)) (83) Project [codegen id : 1] Output [1]: [d_week_seq#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index ab4c4ad4ae65b..dd37b8801e957 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -467,7 +467,7 @@ Input [2]: [d_date#8, d_week_seq#9] (82) Filter [codegen id : 1] Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_date#8) AND (d_date#8 = 2000-01-03)) +Condition : (isnotnull(d_date#8) AND (d_date#8 = 10959)) (83) Project [codegen id : 1] Output [1]: [d_week_seq#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt index 2466a4d38f21e..aec81c0d5c7aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt @@ -47,7 +47,7 @@ Input [5]: [sr_item_sk#1, sr_reason_sk#2, sr_ticket_number#3, sr_return_quantity Output [2]: [r_reason_sk#6, r_reason_desc#7] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] +PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28), IsNotNull(r_reason_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -55,7 +55,7 @@ Input [2]: [r_reason_sk#6, r_reason_desc#7] (7) Filter [codegen id : 1] Input [2]: [r_reason_sk#6, r_reason_desc#7] -Condition : ((isnotnull(r_reason_desc#7) AND (r_reason_desc#7 = reason 28 )) AND isnotnull(r_reason_sk#6)) +Condition : ((isnotnull(r_reason_desc#7) AND (r_reason_desc#7 = reason 28)) AND isnotnull(r_reason_sk#6)) (8) Project [codegen id : 1] Output [1]: [r_reason_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index 1f3470198cd20..1875c5418a890 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -85,7 +85,7 @@ Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, s Output [2]: [r_reason_sk#14, r_reason_desc#15] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] +PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28), IsNotNull(r_reason_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 5] @@ -93,7 +93,7 @@ Input [2]: [r_reason_sk#14, r_reason_desc#15] (16) Filter [codegen id : 5] Input [2]: [r_reason_sk#14, r_reason_desc#15] -Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28 )) AND isnotnull(r_reason_sk#14)) +Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28)) AND isnotnull(r_reason_sk#14)) (17) Project [codegen id : 5] Output [1]: [r_reason_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index c77590bf71044..bee7110ecd6dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -175,7 +175,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Output [2]: [web_site_sk#20, web_company_name#21] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 10] @@ -183,7 +183,7 @@ Input [2]: [web_site_sk#20, web_company_name#21] (31) Filter [codegen id : 10] Input [2]: [web_site_sk#20, web_company_name#21] -Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri )) AND isnotnull(web_site_sk#20)) +Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri)) AND isnotnull(web_site_sk#20)) (32) Project [codegen id : 10] Output [1]: [web_site_sk#20] @@ -214,7 +214,7 @@ Input [2]: [d_date_sk#23, d_date#24] (38) Filter [codegen id : 11] Input [2]: [d_date_sk#23, d_date#24] -Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 1999-02-01)) AND (d_date#24 <= 1999-04-02)) AND isnotnull(d_date_sk#23)) +Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 10623)) AND (d_date#24 <= 10683)) AND isnotnull(d_date_sk#23)) (39) Project [codegen id : 11] Output [1]: [d_date_sk#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 9558a01423452..efa09c2f625cd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -152,7 +152,7 @@ Input [2]: [d_date_sk#17, d_date#18] (24) Filter [codegen id : 9] Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-01)) AND (d_date#18 <= 1999-04-02)) AND isnotnull(d_date_sk#17)) +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 10623)) AND (d_date#18 <= 10683)) AND isnotnull(d_date_sk#17)) (25) Project [codegen id : 9] Output [1]: [d_date_sk#17] @@ -206,7 +206,7 @@ Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_ Output [2]: [web_site_sk#23, web_company_name#24] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 11] @@ -214,7 +214,7 @@ Input [2]: [web_site_sk#23, web_company_name#24] (38) Filter [codegen id : 11] Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri )) AND isnotnull(web_site_sk#23)) +Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri)) AND isnotnull(web_site_sk#23)) (39) Project [codegen id : 11] Output [1]: [web_site_sk#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt index 320a93e19bb27..725939ef24f1b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt @@ -252,7 +252,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Output [2]: [web_site_sk#24, web_company_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 19] @@ -260,7 +260,7 @@ Input [2]: [web_site_sk#24, web_company_name#25] (46) Filter [codegen id : 19] Input [2]: [web_site_sk#24, web_company_name#25] -Condition : ((isnotnull(web_company_name#25) AND (web_company_name#25 = pri )) AND isnotnull(web_site_sk#24)) +Condition : ((isnotnull(web_company_name#25) AND (web_company_name#25 = pri)) AND isnotnull(web_site_sk#24)) (47) Project [codegen id : 19] Output [1]: [web_site_sk#24] @@ -291,7 +291,7 @@ Input [2]: [d_date_sk#27, d_date#28] (53) Filter [codegen id : 20] Input [2]: [d_date_sk#27, d_date#28] -Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 1999-02-01)) AND (d_date#28 <= 1999-04-02)) AND isnotnull(d_date_sk#27)) +Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 10623)) AND (d_date#28 <= 10683)) AND isnotnull(d_date_sk#27)) (54) Project [codegen id : 20] Output [1]: [d_date_sk#27] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index 2f719a4be1630..9f15375b5cfc0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -229,7 +229,7 @@ Input [2]: [d_date_sk#21, d_date#22] (39) Filter [codegen id : 18] Input [2]: [d_date_sk#21, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-01)) AND (d_date#22 <= 1999-04-02)) AND isnotnull(d_date_sk#21)) +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 10623)) AND (d_date#22 <= 10683)) AND isnotnull(d_date_sk#21)) (40) Project [codegen id : 18] Output [1]: [d_date_sk#21] @@ -283,7 +283,7 @@ Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_ Output [2]: [web_site_sk#27, web_company_name#28] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 20] @@ -291,7 +291,7 @@ Input [2]: [web_site_sk#27, web_company_name#28] (53) Filter [codegen id : 20] Input [2]: [web_site_sk#27, web_company_name#28] -Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri )) AND isnotnull(web_site_sk#27)) +Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri)) AND isnotnull(web_site_sk#27)) (54) Project [codegen id : 20] Output [1]: [web_site_sk#27] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt index 5374a708a0295..e04148fad545e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt @@ -173,7 +173,7 @@ Input [2]: [d_date_sk#25, d_date#26] (18) Filter [codegen id : 4] Input [2]: [d_date_sk#25, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 1998-08-04)) AND (d_date#26 <= 1998-08-18)) AND isnotnull(d_date_sk#25)) +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 10442)) AND (d_date#26 <= 10456)) AND isnotnull(d_date_sk#25)) (19) Project [codegen id : 4] Output [1]: [d_date_sk#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 89362fb4f0efe..36b1ff63b2065 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -143,7 +143,7 @@ Input [2]: [d_date_sk#22, d_date#23] (12) Filter [codegen id : 3] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 10442)) AND (d_date#23 <= 10456)) AND isnotnull(d_date_sk#22)) (13) Project [codegen id : 3] Output [1]: [d_date_sk#22]